Skip to content
Closed
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 1
  • Loading branch information
brkyvz committed Nov 10, 2015
commit 6d5e50e1c972ea660f13a3e44e8524eeb7ef5a5d
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,14 @@ package org.apache.spark.streaming.util

import java.io._
import java.nio.ByteBuffer
import java.util.concurrent.{ExecutionException, ThreadPoolExecutor}
import java.util.concurrent.atomic.AtomicInteger
import java.util.{Iterator => JIterator}
import java.util.concurrent.ThreadPoolExecutor

import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.concurrent._
import scala.concurrent.duration._
import scala.language.{implicitConversions, postfixOps}
import scala.util.{Failure, Success}

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand All @@ -37,13 +36,12 @@ import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.AsyncAssertions.Waiter
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfter}
import org.scalatest.mock.MockitoSugar

import org.apache.spark.streaming.scheduler._
import org.apache.spark.util.{ThreadUtils, ManualClock, Utils}
import org.apache.spark.{SparkException, SparkConf, SparkFunSuite}
import org.apache.spark.{SparkConf, SparkFunSuite}

/** Common tests for WriteAheadLogs that we would like to test with different configurations. */
abstract class CommonWriteAheadLogTests(
Expand Down Expand Up @@ -367,7 +365,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
}

// we make the write requests in separate threads so that we don't block the test thread
private def promiseWriteEvent(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = {
private def writeAsync(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = {
val p = Promise[Unit]()
p.completeWith(Future {
val v = wal.write(event, time)
Expand All @@ -376,30 +374,9 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
p
}

/**
* In order to block the writes on the writer thread, we mock the write method, and block it
* for some time with a promise.
*/
private def writeBlockingPromise(wal: WriteAheadLog): (Promise[Any], Waiter) = {
// we would like to block the write so that we can queue requests
val promise = Promise[Any]()
val waiter = new Waiter
when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
new Answer[WriteAheadLogRecordHandle] {
override def answer(invocation: InvocationOnMock): WriteAheadLogRecordHandle = {
waiter.dismiss()
Await.ready(promise.future, 4.seconds)
walHandle
}
}
)
(promise, waiter)
}

test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)
// block the write so that we can batch some records
val (promise, waiter) = writeBlockingPromise(wal)
val blockingWal = new BlockingWriteAheadLog(wal, walHandle)
val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf)

val event1 = "hello"
val event2 = "world"
Expand All @@ -409,17 +386,19 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(

// The queue.take() immediately takes the 3, and there is nothing left in the queue at that
// moment. Then the promise blocks the writing of 3. The rest get queued.
promiseWriteEvent(batchedWal, event1, 3L)
waiter.await()
writeAsync(batchedWal, event1, 3L)
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
Copy link
Contributor

Choose a reason for hiding this comment

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

grammar:it takes time for 3 to get written

promiseWriteEvent(batchedWal, event2, 5L)
promiseWriteEvent(batchedWal, event3, 8L)
promiseWriteEvent(batchedWal, event4, 12L)
promiseWriteEvent(batchedWal, event5, 10L)
writeAsync(batchedWal, event2, 5L)
writeAsync(batchedWal, event3, 8L)
writeAsync(batchedWal, event4, 12L)
writeAsync(batchedWal, event5, 10L)
eventually(timeout(1 second)) {
assert(walBatchingThreadPool.getActiveCount === 5)
}
promise.success(true)
blockingWal.allowWrite()

val buffer1 = wrapArrayArrayByte(Array(event1))
val buffer2 = wrapArrayArrayByte(Array(event2, event3, event4, event5))
Expand All @@ -441,10 +420,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
}

test("BatchedWriteAheadLog - fail everything in queue during shutdown") {
val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)

// block the write so that we can batch some records
writeBlockingPromise(wal)
val blockingWal = new BlockingWriteAheadLog(wal, walHandle)
val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf)

val event1 = ("hello", 3L)
val event2 = ("world", 5L)
Expand All @@ -455,7 +432,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
// The queue.take() immediately takes the 3, and there is nothing left in the queue at that
// moment. Then the promise blocks the writing of 3. The rest get queued.
val writePromises = Seq(event1, event2, event3, event4, event5).map { event =>
promiseWriteEvent(batchedWal, event._1, event._2)
writeAsync(batchedWal, event._1, event._2)
}

eventually(timeout(1 second)) {
Expand Down Expand Up @@ -645,4 +622,35 @@ object WriteAheadLogSuite {
def wrapArrayArrayByte[T](records: Array[T]): ByteBuffer = {
ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(Utils.serialize[T])))
}

/**
* A wrapper WriteAheadLog that blocks the write function to allow batching with the
* BatchedWriteAheadLog.
*/
class BlockingWriteAheadLog(
wal: WriteAheadLog,
handle: WriteAheadLogRecordHandle) extends WriteAheadLog {
@volatile private var isWriteCalled: Boolean = false
@volatile private var blockWrite: Boolean = true

override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = {
isWriteCalled = true
eventually(Eventually.timeout(2 second)) {
assert(!blockWrite)
}
wal.write(record, time)
isWriteCalled = false
handle
}
override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = ???
override def readAll(): JIterator[ByteBuffer] = ???
override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = ???
override def close(): Unit = wal.close()

def allowWrite(): Unit = {
blockWrite = false
}

def isBlocked: Boolean = isWriteCalled
}
}