Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Address comment
  • Loading branch information
brkyvz committed Nov 11, 2015
commit c3c177d36caa711f0da646c6057d9ccf847165ff
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,9 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp
buffer.clear()
}
}

/** Method for querying the queue length. Should only be used in tests. */
private def getQueueLength(): Int = walWriteQueue.size()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do you need this function? Can you use the private method tester to directly access walWriteQueue?

}

/** Static methods for aggregating and de-aggregating records. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.Eventually._
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfter}
import org.scalatest.{PrivateMethodTester, BeforeAndAfterEach, BeforeAndAfter}
import org.scalatest.mock.MockitoSugar

import org.apache.spark.streaming.scheduler._
Expand Down Expand Up @@ -314,7 +314,11 @@ class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
allowBatching = true,
closeFileAfterWrite = false,
"BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with Eventually {
"BatchedWriteAheadLog")
with MockitoSugar
with BeforeAndAfterEach
with Eventually
with PrivateMethodTester {

import BatchedWriteAheadLog._
import WriteAheadLogSuite._
Expand All @@ -325,6 +329,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
private var walBatchingExecutionContext: ExecutionContextExecutorService = _
private val sparkConf = new SparkConf()

private val queueLength = PrivateMethod[Int]('getQueueLength)

override def beforeEach(): Unit = {
wal = mock[WriteAheadLog]
walHandle = mock[WriteAheadLogRecordHandle]
Expand Down Expand Up @@ -390,20 +396,22 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
eventually(timeout(1 second)) {
assert(blockingWal.isBlocked)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also add checks that the batchedWALs internal queue lengths? Here is should be 0, and in line 399 it will be 5.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add assert(batchedWal.invokePrivate(queueLength()) === 0)

}
// rest of the records will be batched while it takes 3 to get written
// rest of the records will be batched while it takes time for 3 to get written
writeAsync(batchedWal, event2, 5L)
writeAsync(batchedWal, event3, 8L)
writeAsync(batchedWal, event4, 12L)
writeAsync(batchedWal, event5, 10L)
eventually(timeout(1 second)) {
assert(walBatchingThreadPool.getActiveCount === 5)
assert(batchedWal.invokePrivate(queueLength()) === 4)
}
blockingWal.allowWrite()

val buffer1 = wrapArrayArrayByte(Array(event1))
val buffer2 = wrapArrayArrayByte(Array(event2, event3, event4, event5))

eventually(timeout(1 second)) {
assert(batchedWal.invokePrivate(queueLength()) === 0)
verify(wal, times(1)).write(meq(buffer1), meq(3L))
// the file name should be the timestamp of the last record, as events should be naturally
// in order of timestamp, and we need the last element.
Expand Down