From 8f92f10f4aa17b1a19a72e1c257273bb26080bb5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 13 Oct 2015 13:40:37 -0700 Subject: [PATCH 01/26] ready for testing --- .../scheduler/ReceivedBlockTracker.scala | 217 +++++++++++++++--- .../streaming/scheduler/ReceiverTracker.scala | 18 +- .../streaming/util/WriteAheadLogUtils.scala | 6 + .../streaming/ReceivedBlockTrackerSuite.scala | 81 +++++-- 4 files changed, 269 insertions(+), 53 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index f2711d1355e60..99b0c6be7458f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -18,9 +18,11 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration @@ -41,7 +43,6 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent - /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] case class AllocatedBlocks(streamIdToAllocatedBlocks: Map[Int, Seq[ReceivedBlockInfo]]) { @@ -73,6 +74,21 @@ private[streaming] class ReceivedBlockTracker( private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue] private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] private val writeAheadLogOption = createWriteAheadLog() + private val walWriteQueue = new ConcurrentLinkedQueue[ReceivedBlockTrackerLogEvent]() + + private trait WALWriteStatus + private object Pending extends WALWriteStatus + private object Success extends WALWriteStatus + private object Fail extends WALWriteStatus + + // stores the status for wal writes added to the queue + private val walWriteStatusMap = + new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, WALWriteStatus]() + + private val WAL_WRITE_STATUS_CHECK_BACKOFF = 10 // 10 millis + private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds + + private val writeAheadLogBatchWriter: Option[BatchLogWriter] = createBatchWriteAheadLogWriter() private var lastAllocatedBatchTime: Time = null @@ -82,13 +98,20 @@ private[streaming] class ReceivedBlockTracker( } /** Add received block. This event will get written to the write ahead log (if enabled). */ - def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { + private def addBlock0( + receivedBlockInfo: ReceivedBlockInfo, + writeOp: ReceivedBlockTrackerLogEvent => Boolean): Boolean = { try { - writeToLog(BlockAdditionEvent(receivedBlockInfo)) - getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo - logDebug(s"Stream ${receivedBlockInfo.streamId} received " + - s"block ${receivedBlockInfo.blockStoreResult.blockId}") - true + val result = writeOp(BlockAdditionEvent(receivedBlockInfo)) + if (result) { + afterBlockAddAcknowledged(receivedBlockInfo) + logDebug(s"Stream ${receivedBlockInfo.streamId} received " + + s"block ${receivedBlockInfo.blockStoreResult.blockId}") + } else { + logDebug(s"Failed to acknowledge stream ${receivedBlockInfo.streamId} receiving " + + s"block ${receivedBlockInfo.blockStoreResult.blockId} in the Write Ahead Log.") + } + result } catch { case e: Exception => logError(s"Error adding block $receivedBlockInfo", e) @@ -96,20 +119,50 @@ private[streaming] class ReceivedBlockTracker( } } + /** Add received block. This event will get written to the write ahead log (if enabled). */ + def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { + addBlock0(receivedBlockInfo, writeToLog) + } + + private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = { + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + } + + def addBlockAsync(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + addBlock0(receivedBlockInfo, writeToLogAsync) + } + + def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { + if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled + walWriteStatusMap.put(event, Pending) + walWriteQueue.offer(event) + var timedOut = false + val start = clock.getTimeMillis() + while (walWriteStatusMap.get(event) == Pending) { + Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) + if (clock.getTimeMillis() - WAL_WRITE_STATUS_TIMEOUT > start) timedOut = true + } + walWriteStatusMap.remove(event) == Success + } + /** * Allocate all unallocated blocks to the given batch. * This event will get written to the write ahead log (if enabled). */ - def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { + private def allocateBlocksToBatch0( + batchTime: Time, + writeOp: ReceivedBlockTrackerLogEvent => Boolean): Unit = { if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { val streamIdToBlocks = streamIds.map { streamId => (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) - writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks)) - timeToAllocatedBlocks(batchTime) = allocatedBlocks - lastAllocatedBatchTime = batchTime - allocatedBlocks + val result = writeOp(BatchAllocationEvent(batchTime, allocatedBlocks)) + if (result) { + afterBatchAllocationAcknowledged(batchTime, allocatedBlocks) + } else { + logInfo(s"Possibly processed batch $batchTime need to be processed again in WAL recovery") + } } else { // This situation occurs when: // 1. WAL is ended with BatchAllocationEvent, but without BatchCleanupEvent, @@ -122,6 +175,25 @@ private[streaming] class ReceivedBlockTracker( } } + /** + * Allocate all unallocated blocks to the given batch. + * This event will get written to the write ahead log (if enabled). + */ + def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { + allocateBlocksToBatch0(batchTime, writeToLog) + } + + def allocateBlocksToBatchAsync(batchTime: Time): Unit = { + allocateBlocksToBatch0(batchTime, writeToLogAsync) + } + + private def afterBatchAllocationAcknowledged( + batchTime: Time, + allocatedBlocks: AllocatedBlocks): Unit = { + timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + lastAllocatedBatchTime = batchTime + } + /** Get the blocks allocated to the given batch. */ def getBlocksOfBatch(batchTime: Time): Map[Int, Seq[ReceivedBlockInfo]] = synchronized { timeToAllocatedBlocks.get(batchTime).map { _.streamIdToAllocatedBlocks }.getOrElse(Map.empty) @@ -153,17 +225,47 @@ private[streaming] class ReceivedBlockTracker( * Clean up block information of old batches. If waitForCompletion is true, this method * returns only after the files are cleaned up. */ - def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { + private def cleanupOldBatches0( + cleanupThreshTime: Time, + waitForCompletion: Boolean, + writeOp: ReceivedBlockTrackerLogEvent => Boolean): Unit = { require(cleanupThreshTime.milliseconds < clock.getTimeMillis()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq logInfo("Deleting batches " + timesToCleanup) - writeToLog(BatchCleanupEvent(timesToCleanup)) + if (writeOp(BatchCleanupEvent(timesToCleanup))) { + afterBatchCleanupAcknowledged(cleanupThreshTime, waitForCompletion, timesToCleanup) + } else { + logWarning("Failed to acknowledge batch clean up in the Write Ahead Log.") + } + } + + /** + * Clean up block information of old batches. If waitForCompletion is true, this method + * returns only after the files are cleaned up. + */ + def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { + cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLog) + } + + /** + * Clean up block information of old batches. If waitForCompletion is true, this method + * returns only after the files are cleaned up. + */ + def cleanupOldBatchesAsync(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = { + cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLogAsync) + } + + private def afterBatchCleanupAcknowledged( + cleanupThreshTime: Time, + waitForCompletion: Boolean, + timesToCleanup: Seq[Time]): Unit = { timeToAllocatedBlocks --= timesToCleanup writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) } /** Stop the block tracker. */ def stop() { + writeAheadLogBatchWriter.foreach { _.stop() } writeAheadLogOption.foreach { _.close() } } @@ -176,7 +278,7 @@ private[streaming] class ReceivedBlockTracker( def insertAddedBlock(receivedBlockInfo: ReceivedBlockInfo) { logTrace(s"Recovery: Inserting added block $receivedBlockInfo") receivedBlockInfo.setBlockIdInvalid() - getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + afterBlockAddAcknowledged(receivedBlockInfo) } // Insert the recovered block-to-batch allocations and clear the queue of received blocks @@ -185,8 +287,7 @@ private[streaming] class ReceivedBlockTracker( logTrace(s"Recovery: Inserting allocated batch for time $batchTime to " + s"${allocatedBlocks.streamIdToAllocatedBlocks}") streamIdToUnallocatedBlockQueues.values.foreach { _.clear() } - lastAllocatedBatchTime = batchTime - timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + afterBatchAllocationAcknowledged(batchTime, allocatedBlocks) } // Cleanup the batch allocations @@ -195,31 +296,42 @@ private[streaming] class ReceivedBlockTracker( timeToAllocatedBlocks --= batchTimes } + def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { + event match { + case BlockAdditionEvent(receivedBlockInfo) => + insertAddedBlock(receivedBlockInfo) + case BatchAllocationEvent(time, allocatedBlocks) => + insertAllocatedBatch(time, allocatedBlocks) + case BatchCleanupEvent(batchTimes) => + cleanupBatches(batchTimes) + } + } + writeAheadLogOption.foreach { writeAheadLog => logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") writeAheadLog.readAll().asScala.foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) - Utils.deserialize[ReceivedBlockTrackerLogEvent]( - byteBuffer.array, Thread.currentThread().getContextClassLoader) match { - case BlockAdditionEvent(receivedBlockInfo) => - insertAddedBlock(receivedBlockInfo) - case BatchAllocationEvent(time, allocatedBlocks) => - insertAllocatedBatch(time, allocatedBlocks) - case BatchCleanupEvent(batchTimes) => - cleanupBatches(batchTimes) + try { + Utils.deserialize[List[ReceivedBlockTrackerLogEvent]]( + byteBuffer.array, Thread.currentThread().getContextClassLoader).foreach(resolveEvent) + } catch { + case e: ClassCastException => + resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent]( + byteBuffer.array, Thread.currentThread().getContextClassLoader)) } } } } /** Write an update to the tracker to the write ahead log */ - private def writeToLog(record: ReceivedBlockTrackerLogEvent) { + private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { logDebug(s"Writing to log $record") writeAheadLogOption.foreach { logManager => logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) } } + true } /** Get the queue of received blocks belonging to a particular stream */ @@ -235,8 +347,61 @@ private[streaming] class ReceivedBlockTracker( } } + private def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = { + if (!WriteAheadLogUtils.isBatchingEnabled(conf)) return None + val writer = checkpointDirOption.map(_ => new BatchLogWriter) + writer.foreach { runnable => + new Thread(runnable, "Batch WAL Writer").start() + Runtime.getRuntime.addShutdownHook(new Thread { + override def run(): Unit = runnable.stop() + }) + } + writer + } + /** Check if the write ahead log is enabled. This is only used for testing purposes. */ private[streaming] def isWriteAheadLogEnabled: Boolean = writeAheadLogOption.nonEmpty + + private class BatchLogWriter extends Runnable { + + var active: Boolean = true + + private def writeRecords(records: List[ReceivedBlockTrackerLogEvent]): Unit = { + writeAheadLogOption.foreach { logManager => + if (records.nonEmpty) { + logDebug(s"Writing to log $records") + logManager.write(ByteBuffer.wrap(Utils.serialize(records)), clock.getTimeMillis()) + } + } + } + + def stop(): Unit = { + active = false + } + + private def flushRecords(): Unit = { + val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]() + while (!walWriteQueue.isEmpty) { + buffer.append(walWriteQueue.poll()) + } + val records = buffer.toList + try { + writeRecords(records) + records.foreach(walWriteStatusMap.put(_, Success)) + } catch { + case e: Exception => + logWarning(s"Batch WAL Writer failed to write $records") + records.foreach(walWriteStatusMap.put(_, Fail)) + } + } + + override def run(): Unit = { + while (active) { + flushRecords() + Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) + } + } + } } private[streaming] object ReceivedBlockTracker { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index d053e9e84910f..a0fbd88d72b3c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -198,7 +198,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Allocate all unallocated blocks to the given batch. */ def allocateBlocksToBatch(batchTime: Time): Unit = { if (receiverInputStreams.nonEmpty) { - receivedBlockTracker.allocateBlocksToBatch(batchTime) + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { + receivedBlockTracker.allocateBlocksToBatchAsync(batchTime) + } else { + receivedBlockTracker.allocateBlocksToBatch(batchTime) + } } } @@ -218,7 +222,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false */ def cleanupOldBlocksAndBatches(cleanupThreshTime: Time) { // Clean up old block and batch metadata - receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { + receivedBlockTracker.cleanupOldBatchesAsync(cleanupThreshTime, waitForCompletion = false) + } else { + receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) + } // Signal the receivers to delete old block data if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { @@ -310,7 +318,11 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Add new blocks for the given stream */ private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - receivedBlockTracker.addBlock(receivedBlockInfo) + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { + receivedBlockTracker.addBlockAsync(receivedBlockInfo) + } else { + receivedBlockTracker.addBlock(receivedBlockInfo) + } } /** Report error sent by a receiver */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 7f6ff12c58d47..6485f6a1a1e9a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -36,6 +36,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { val DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY = "spark.streaming.driver.writeAheadLog.rollingIntervalSecs" val DRIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.driver.writeAheadLog.maxFailures" + val DRIVER_WAL_BATCHING_CONF_KEY = "spark.streaming.driver.writeAheadLog.enableBatching" val DEFAULT_ROLLING_INTERVAL_SECS = 60 val DEFAULT_MAX_FAILURES = 3 @@ -60,6 +61,11 @@ private[streaming] object WriteAheadLogUtils extends Logging { } } + def isBatchingEnabled(conf: SparkConf): Boolean = { + conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) || sys.props.getOrElse( + DRIVER_WAL_BATCHING_CONF_KEY, "false").toBoolean + } + /** * Create a WriteAheadLog for the driver. If configured with custom WAL class, it will try * to create instance of that class, otherwise it will create the default FileBasedWriteAheadLog. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index f793a12843b2f..762bc0b0d6d2b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadL import org.apache.spark.streaming.util.WriteAheadLogSuite._ import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} -class ReceivedBlockTrackerSuite +abstract class ReceivedBlockTrackerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { val hadoopConf = new Configuration() @@ -46,10 +46,14 @@ class ReceivedBlockTrackerSuite var allReceivedBlockTrackers = new ArrayBuffer[ReceivedBlockTracker]() var checkpointDirectory: File = null var conf: SparkConf = null + val isBatchingEnabled: Boolean before { conf = new SparkConf().setMaster("local[2]").setAppName("ReceivedBlockTrackerSuite") checkpointDirectory = Utils.createTempDir() + if (isBatchingEnabled) { + conf.set("spark.streaming.driver.writeAheadLog.enableBatching", "true") + } } after { @@ -57,38 +61,64 @@ class ReceivedBlockTrackerSuite Utils.deleteRecursively(checkpointDirectory) } + // Generate and add blocks to the given tracker + def addBlockInfos( + tracker: ReceivedBlockTracker, + blockInfos: Seq[ReceivedBlockInfo] = generateBlockInfos()): Seq[ReceivedBlockInfo] = { + if (isBatchingEnabled) { + blockInfos.map(tracker.addBlockAsync) + } else { + blockInfos.map(tracker.addBlock) + } + blockInfos + } + + def allocateBlocksToBatch(t: Time, tracker: ReceivedBlockTracker): Unit = { + if (isBatchingEnabled) { + tracker.allocateBlocksToBatchAsync(t) + } else { + tracker.allocateBlocksToBatch(t) + } + } + + def cleanupOldBatches(t: Time, wait: Boolean, tracker: ReceivedBlockTracker): Unit = { + if (isBatchingEnabled) { + tracker.cleanupOldBatchesAsync(t, wait) + } else { + tracker.cleanupOldBatches(t, wait) + } + } + test("block addition, and block to batch allocation") { val receivedBlockTracker = createTracker(setCheckpointDir = false) receivedBlockTracker.isWriteAheadLogEnabled should be (false) // should be disable by default receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty - val blockInfos = generateBlockInfos() - blockInfos.map(receivedBlockTracker.addBlock) + val blockInfos = addBlockInfos(receivedBlockTracker) // Verify added blocks are unallocated blocks receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos receivedBlockTracker.hasUnallocatedReceivedBlocks should be (true) - // Allocate the blocks to a batch and verify that all of them have been allocated - receivedBlockTracker.allocateBlocksToBatch(1) + allocateBlocksToBatch(1, receivedBlockTracker) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos receivedBlockTracker.getBlocksOfBatch(1) shouldEqual Map(streamId -> blockInfos) receivedBlockTracker.getUnallocatedBlocks(streamId) shouldBe empty receivedBlockTracker.hasUnallocatedReceivedBlocks should be (false) // Allocate no blocks to another batch - receivedBlockTracker.allocateBlocksToBatch(2) + allocateBlocksToBatch(2, receivedBlockTracker) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty receivedBlockTracker.getBlocksOfBatch(2) shouldEqual Map(streamId -> Seq.empty) // Verify that older batches have no operation on batch allocation, // will return the same blocks as previously allocated. - receivedBlockTracker.allocateBlocksToBatch(1) + allocateBlocksToBatch(1, receivedBlockTracker) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos - blockInfos.map(receivedBlockTracker.addBlock) - receivedBlockTracker.allocateBlocksToBatch(2) + addBlockInfos(receivedBlockTracker, blockInfos) + allocateBlocksToBatch(2, receivedBlockTracker) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos } @@ -103,13 +133,6 @@ class ReceivedBlockTrackerSuite manualClock.advance(timeIncrementMillis) } - // Generate and add blocks to the given tracker - def addBlockInfos(tracker: ReceivedBlockTracker): Seq[ReceivedBlockInfo] = { - val blockInfos = generateBlockInfos() - blockInfos.map(tracker.addBlock) - blockInfos - } - // Print the data present in the log ahead files in the log directory def printLogFiles(message: String) { val fileContents = getWriteAheadLogFiles().map { file => @@ -149,10 +172,9 @@ class ReceivedBlockTrackerSuite block.isBlockIdValid() should be (false) } - // Allocate blocks to batch and verify whether the unallocated blocks got allocated val batchTime1 = manualClock.getTimeMillis() - tracker2.allocateBlocksToBatch(batchTime1) + allocateBlocksToBatch(batchTime1, tracker2) tracker2.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 tracker2.getBlocksOfBatch(batchTime1) shouldEqual Map(streamId -> blockInfos1) @@ -160,9 +182,8 @@ class ReceivedBlockTrackerSuite incrementTime() val batchTime2 = manualClock.getTimeMillis() val blockInfos2 = addBlockInfos(tracker2) - tracker2.allocateBlocksToBatch(batchTime2) + allocateBlocksToBatch(batchTime2, tracker2) tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 - // Verify whether log has correct contents val expectedWrittenData2 = expectedWrittenData1 ++ Seq(createBatchAllocation(batchTime1, blockInfos1)) ++ @@ -180,8 +201,7 @@ class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true) - + cleanupOldBatches(batchTime2, true, tracker3) // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty getWrittenLogData(getWriteAheadLogFiles().last) should contain(createBatchCleanup(batchTime1)) @@ -241,8 +261,13 @@ class ReceivedBlockTrackerSuite : Seq[ReceivedBlockTrackerLogEvent] = { logFiles.flatMap { file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq - }.map { byteBuffer => - Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) + }.flatMap { byteBuffer => + try { + Utils.deserialize[List[ReceivedBlockTrackerLogEvent]](byteBuffer.array) + } catch { + case e: ClassCastException => + Seq(Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array)) + } }.toList } @@ -268,3 +293,11 @@ class ReceivedBlockTrackerSuite implicit def timeToMillis(time: Time): Long = time.milliseconds } + +class SyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite { + override val isBatchingEnabled = false +} + +class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite { + override val isBatchingEnabled = true +} From 78c6069477422d4984b7107db435245c811dbab9 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 14 Oct 2015 08:21:10 -0700 Subject: [PATCH 02/26] save changes --- .../scheduler/ReceivedBlockTracker.scala | 29 +++++++++---------- .../streaming/scheduler/ReceiverTracker.scala | 23 ++++++++++----- 2 files changed, 30 insertions(+), 22 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 99b0c6be7458f..3bf19b8dac546 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -132,19 +132,6 @@ private[streaming] class ReceivedBlockTracker( addBlock0(receivedBlockInfo, writeToLogAsync) } - def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { - if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled - walWriteStatusMap.put(event, Pending) - walWriteQueue.offer(event) - var timedOut = false - val start = clock.getTimeMillis() - while (walWriteStatusMap.get(event) == Pending) { - Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) - if (clock.getTimeMillis() - WAL_WRITE_STATUS_TIMEOUT > start) timedOut = true - } - walWriteStatusMap.remove(event) == Success - } - /** * Allocate all unallocated blocks to the given batch. * This event will get written to the write ahead log (if enabled). @@ -326,7 +313,6 @@ private[streaming] class ReceivedBlockTracker( /** Write an update to the tracker to the write ahead log */ private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { - logDebug(s"Writing to log $record") writeAheadLogOption.foreach { logManager => logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) } @@ -334,6 +320,19 @@ private[streaming] class ReceivedBlockTracker( true } + private def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { + if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled + walWriteStatusMap.put(event, Pending) + walWriteQueue.offer(event) + var timedOut = false + val start = clock.getTimeMillis() + while (walWriteStatusMap.get(event) == Pending) { + Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) + if (clock.getTimeMillis() - WAL_WRITE_STATUS_TIMEOUT > start) timedOut = true + } + walWriteStatusMap.remove(event) == Success + } + /** Get the queue of received blocks belonging to a particular stream */ private def getReceivedBlockQueue(streamId: Int): ReceivedBlockQueue = { streamIdToUnallocatedBlockQueues.getOrElseUpdate(streamId, new ReceivedBlockQueue) @@ -369,7 +368,7 @@ private[streaming] class ReceivedBlockTracker( private def writeRecords(records: List[ReceivedBlockTrackerLogEvent]): Unit = { writeAheadLogOption.foreach { logManager => if (records.nonEmpty) { - logDebug(s"Writing to log $records") + logDebug(s"Batched ${records.length} records for WAL write") logManager.write(ByteBuffer.wrap(Utils.serialize(records)), clock.getTimeMillis()) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index a0fbd88d72b3c..76e60aa4c91a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler import java.util.concurrent.{TimeUnit, CountDownLatch} import scala.collection.mutable.HashMap -import scala.concurrent.ExecutionContext +import scala.concurrent.{Future, ExecutionContext} import scala.language.existentials import scala.util.{Failure, Success} @@ -318,11 +318,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Add new blocks for the given stream */ private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { - receivedBlockTracker.addBlockAsync(receivedBlockInfo) - } else { - receivedBlockTracker.addBlock(receivedBlockInfo) - } + receivedBlockTracker.addBlock(receivedBlockInfo) + } + + /** Add new blocks for the given stream */ + private def addBlockAsync(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + receivedBlockTracker.addBlockAsync(receivedBlockInfo) } /** Report error sent by a receiver */ @@ -440,6 +441,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private val submitJobThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool")) + private val walBatchingThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) + override def receive: PartialFunction[Any, Unit] = { // Local messages case StartAllReceivers(receivers) => @@ -489,7 +493,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false registerReceiver(streamId, typ, hostPort, receiverEndpoint, context.senderAddress) context.reply(successful) case AddBlock(receivedBlockInfo) => - context.reply(addBlock(receivedBlockInfo)) + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { + val f = Future(addBlockAsync(receivedBlockInfo))(walBatchingThreadPool) + f.onComplete(result => context.reply(result.get))(walBatchingThreadPool) + } else { + context.reply(addBlock(receivedBlockInfo)) + } case DeregisterReceiver(streamId, message, error) => deregisterReceiver(streamId, message, error) context.reply(true) From ee36f8968354ecb14f1153b9bd7fb8f0d4bb9e1e Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 15 Oct 2015 17:21:56 -0700 Subject: [PATCH 03/26] add more tests --- .../scheduler/ReceivedBlockTracker.scala | 62 ++++++++----- .../streaming/util/WriteAheadLogUtils.scala | 3 +- .../streaming/ReceivedBlockTrackerSuite.scala | 93 ++++++++++++++++++- 3 files changed, 133 insertions(+), 25 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 3bf19b8dac546..cb37a74a56def 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -74,15 +74,11 @@ private[streaming] class ReceivedBlockTracker( private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue] private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] private val writeAheadLogOption = createWriteAheadLog() - private val walWriteQueue = new ConcurrentLinkedQueue[ReceivedBlockTrackerLogEvent]() + // exposed for tests + protected val walWriteQueue = new ConcurrentLinkedQueue[ReceivedBlockTrackerLogEvent]() - private trait WALWriteStatus - private object Pending extends WALWriteStatus - private object Success extends WALWriteStatus - private object Fail extends WALWriteStatus - - // stores the status for wal writes added to the queue - private val walWriteStatusMap = + // stores the status for wal writes added to the queue. Exposed for tests + protected val walWriteStatusMap = new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, WALWriteStatus]() private val WAL_WRITE_STATUS_CHECK_BACKOFF = 10 // 10 millis @@ -119,13 +115,14 @@ private[streaming] class ReceivedBlockTracker( } } - /** Add received block. This event will get written to the write ahead log (if enabled). */ - def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { - addBlock0(receivedBlockInfo, writeToLog) + /** Update in-memory state after block add event has been logged to WAL. */ + private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = synchronized { + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo } - private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = { - getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + /** Add received block. This event will get written to the write ahead log (if enabled). */ + def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + addBlock0(receivedBlockInfo, writeToLog) } def addBlockAsync(receivedBlockInfo: ReceivedBlockInfo): Boolean = { @@ -166,7 +163,7 @@ private[streaming] class ReceivedBlockTracker( * Allocate all unallocated blocks to the given batch. * This event will get written to the write ahead log (if enabled). */ - def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { + def allocateBlocksToBatch(batchTime: Time): Unit = { allocateBlocksToBatch0(batchTime, writeToLog) } @@ -174,9 +171,10 @@ private[streaming] class ReceivedBlockTracker( allocateBlocksToBatch0(batchTime, writeToLogAsync) } + /** Update in-memory state after batch allocation event has been logged to WAL. */ private def afterBatchAllocationAcknowledged( batchTime: Time, - allocatedBlocks: AllocatedBlocks): Unit = { + allocatedBlocks: AllocatedBlocks): Unit = synchronized { timeToAllocatedBlocks.put(batchTime, allocatedBlocks) lastAllocatedBatchTime = batchTime } @@ -230,22 +228,24 @@ private[streaming] class ReceivedBlockTracker( * Clean up block information of old batches. If waitForCompletion is true, this method * returns only after the files are cleaned up. */ - def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { + def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = { cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLog) } /** * Clean up block information of old batches. If waitForCompletion is true, this method - * returns only after the files are cleaned up. + * returns only after the files are cleaned up. Not really async, the event will be written to + * a WAL within a batch, if WAL's are enabled. */ def cleanupOldBatchesAsync(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = { cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLogAsync) } + /** Update in-memory state after clean up event has been logged to WAL. */ private def afterBatchCleanupAcknowledged( cleanupThreshTime: Time, waitForCompletion: Boolean, - timesToCleanup: Seq[Time]): Unit = { + timesToCleanup: Seq[Time]): Unit = synchronized { timeToAllocatedBlocks --= timesToCleanup writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) } @@ -314,13 +314,20 @@ private[streaming] class ReceivedBlockTracker( private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { writeAheadLogOption.foreach { logManager => + logTrace(s"Writing record: $record") logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) } } true } - private def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { + import WALWriteStatus._ + + /** + * Adds LogEvents to a queue so that they can be batched and written to the WAL. + * Exposed for testing. + */ + private[streaming] def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled walWriteStatusMap.put(event, Pending) walWriteQueue.offer(event) @@ -346,7 +353,11 @@ private[streaming] class ReceivedBlockTracker( } } - private def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = { + /** + * Creates a WAL Writer in a separate thread to enable batching of log events. + * Exposed for tests. + */ + protected def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = { if (!WriteAheadLogUtils.isBatchingEnabled(conf)) return None val writer = checkpointDirOption.map(_ => new BatchLogWriter) writer.foreach { runnable => @@ -361,7 +372,8 @@ private[streaming] class ReceivedBlockTracker( /** Check if the write ahead log is enabled. This is only used for testing purposes. */ private[streaming] def isWriteAheadLogEnabled: Boolean = writeAheadLogOption.nonEmpty - private class BatchLogWriter extends Runnable { + /** A helper class that writes LogEvents in a separate thread to allow for batching. */ + private[streaming] class BatchLogWriter extends Runnable { var active: Boolean = true @@ -408,3 +420,11 @@ private[streaming] object ReceivedBlockTracker { new Path(checkpointDir, "receivedBlockMetadata").toString } } + +/** State of LogEvents during batching. */ +private[streaming] trait WALWriteStatus +private[streaming] object WALWriteStatus { + private[streaming] case object Pending extends WALWriteStatus + private[streaming] case object Success extends WALWriteStatus + private[streaming] case object Fail extends WALWriteStatus +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 6485f6a1a1e9a..caf50479e0b0f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -62,8 +62,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { } def isBatchingEnabled(conf: SparkConf): Boolean = { - conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) || sys.props.getOrElse( - DRIVER_WAL_BATCHING_CONF_KEY, "false").toBoolean + conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 762bc0b0d6d2b..8a5015c6b30c1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -18,13 +18,16 @@ package org.apache.spark.streaming import java.io.File +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{Future, ExecutionContext} import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} import scala.util.Random import org.apache.hadoop.conf.Configuration +import org.scalatest.mock.MockitoSugar import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ @@ -34,7 +37,7 @@ import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadLogReader} import org.apache.spark.streaming.util.WriteAheadLogSuite._ -import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} +import org.apache.spark.util._ abstract class ReceivedBlockTrackerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { @@ -298,6 +301,92 @@ class SyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite { override val isBatchingEnabled = false } -class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite { +class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with MockitoSugar { override val isBatchingEnabled = true + + /** Class that will help us test batching. */ + private class TestRBT( + clock: Clock = new SystemClock, + cpDirOption: Option[String] = Some(checkpointDirectory.toString), + recoverFromWriteAheadLog: Boolean = true) + extends ReceivedBlockTracker(conf, hadoopConf, Seq(streamId), clock, + recoverFromWriteAheadLog, cpDirOption) { + + override def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = None + + def updateWALWriteStatus(event: ReceivedBlockTrackerLogEvent, status: WALWriteStatus): Unit = { + walWriteStatusMap.put(event, status) + walWriteQueue.poll() + } + + def getQueueLength(): Int = walWriteQueue.size() + + def addToQueue(event: ReceivedBlockTrackerLogEvent): Boolean = { + writeToLogAsync(event) + } + } + + private def waitUntilTrue(f: () => Int, value: Int): Boolean = { + val timeOut = 2000 + val start = System.currentTimeMillis() + var result = false + while (!result && (System.currentTimeMillis() - start) < timeOut) { + Thread.sleep(50) + result = f() == value + } + result + } + + import WALWriteStatus._ + + test("records get added to a queue") { + val numSuccess = new AtomicInteger() + val numFail = new AtomicInteger() + val rbt = new TestRBT() + + def getNumSuccess(): Int = numSuccess.get() + def getNumFail(): Int = numFail.get() + + val walBatchingThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) + + def eventFuture(event: ReceivedBlockTrackerLogEvent): Unit = { + val f = Future(rbt.addToQueue(event))(walBatchingThreadPool) + f.onSuccess{ case v => + if (v) numSuccess.incrementAndGet() else numFail.incrementAndGet() + }(walBatchingThreadPool) + } + + assert(rbt.getQueueLength === 0) + val event1 = BlockAdditionEvent(ReceivedBlockInfo(1, None, None, null)) + val event2 = BlockAdditionEvent(null) + val event3 = BatchAllocationEvent(null, null) + val event4 = BlockAdditionEvent(ReceivedBlockInfo(2, None, None, null)) + val event5 = BatchCleanupEvent(Nil) + + eventFuture(event1) + assert(waitUntilTrue(rbt.getQueueLength, 1)) + assert(numSuccess.get() === 0) + assert(numFail.get() === 0) + + rbt.updateWALWriteStatus(event1, Fail) + assert(waitUntilTrue(getNumFail, 1)) + assert(waitUntilTrue(rbt.getQueueLength, 0)) + + eventFuture(event2) + eventFuture(event3) + eventFuture(event4) + assert(waitUntilTrue(rbt.getQueueLength, 3)) + rbt.updateWALWriteStatus(event2, Success) + rbt.updateWALWriteStatus(event3, Success) + assert(waitUntilTrue(getNumSuccess, 2)) + assert(waitUntilTrue(rbt.getQueueLength, 1)) + + eventFuture(event5) + assert(waitUntilTrue(rbt.getQueueLength, 2)) + rbt.updateWALWriteStatus(event4, Success) + rbt.updateWALWriteStatus(event5, Success) + assert(waitUntilTrue(getNumSuccess, 4)) + assert(waitUntilTrue(rbt.getQueueLength, 0)) + } } From cf1ad3d2c21c9019259793661020a3f29d1bca25 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 15 Oct 2015 17:41:37 -0700 Subject: [PATCH 04/26] add the combinedRBTLE --- .../scheduler/ReceivedBlockTracker.scala | 16 +++++++--------- .../streaming/ReceivedBlockTrackerSuite.scala | 8 +++----- 2 files changed, 10 insertions(+), 14 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index cb37a74a56def..bcbc1ac8842a3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -42,6 +42,8 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: extends ReceivedBlockTrackerLogEvent private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent +private[streaming] case class CombinedRBTLogEvent(events: List[ReceivedBlockTrackerLogEvent]) + extends ReceivedBlockTrackerLogEvent /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] @@ -285,6 +287,7 @@ private[streaming] class ReceivedBlockTracker( def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { event match { + case CombinedRBTLogEvent(events) => events.foreach(resolveEvent) case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => @@ -298,14 +301,8 @@ private[streaming] class ReceivedBlockTracker( logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") writeAheadLog.readAll().asScala.foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) - try { - Utils.deserialize[List[ReceivedBlockTrackerLogEvent]]( - byteBuffer.array, Thread.currentThread().getContextClassLoader).foreach(resolveEvent) - } catch { - case e: ClassCastException => - resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent]( - byteBuffer.array, Thread.currentThread().getContextClassLoader)) - } + resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent]( + byteBuffer.array, Thread.currentThread().getContextClassLoader)) } } } @@ -381,7 +378,8 @@ private[streaming] class ReceivedBlockTracker( writeAheadLogOption.foreach { logManager => if (records.nonEmpty) { logDebug(s"Batched ${records.length} records for WAL write") - logManager.write(ByteBuffer.wrap(Utils.serialize(records)), clock.getTimeMillis()) + logManager.write(ByteBuffer.wrap(Utils.serialize(CombinedRBTLogEvent(records))), + clock.getTimeMillis()) } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 8a5015c6b30c1..1d6effd6b3391 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -265,11 +265,9 @@ abstract class ReceivedBlockTrackerSuite logFiles.flatMap { file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq }.flatMap { byteBuffer => - try { - Utils.deserialize[List[ReceivedBlockTrackerLogEvent]](byteBuffer.array) - } catch { - case e: ClassCastException => - Seq(Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array)) + Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { + case CombinedRBTLogEvent(events) => events + case others: ReceivedBlockTrackerLogEvent => Seq(others) } }.toList } From 80a0b8d9e994ccf5c9381e12dae4c736ad6c3800 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 27 Oct 2015 23:05:29 -0700 Subject: [PATCH 05/26] addressed most comments --- .../scheduler/ReceivedBlockTracker.scala | 190 +++++++----------- .../streaming/scheduler/ReceiverTracker.scala | 20 +- .../streaming/util/WriteAheadLogUtils.scala | 2 +- .../streaming/ReceivedBlockTrackerSuite.scala | 61 ++---- 4 files changed, 95 insertions(+), 178 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index bcbc1ac8842a3..78ee331c29d16 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -18,19 +18,22 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{ExecutionContext, Await, Promise} +import scala.concurrent.duration._ import scala.language.implicitConversions +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.streaming.Time import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} -import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.util.{ThreadUtils, Clock, Utils} import org.apache.spark.{Logging, SparkConf} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ @@ -42,8 +45,8 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: extends ReceivedBlockTrackerLogEvent private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent -private[streaming] case class CombinedRBTLogEvent(events: List[ReceivedBlockTrackerLogEvent]) - extends ReceivedBlockTrackerLogEvent +private[streaming] case class CombinedReceivedBlockTrackerLogEvent( + events: Seq[ReceivedBlockTrackerLogEvent]) extends ReceivedBlockTrackerLogEvent /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] @@ -77,16 +80,17 @@ private[streaming] class ReceivedBlockTracker( private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] private val writeAheadLogOption = createWriteAheadLog() // exposed for tests - protected val walWriteQueue = new ConcurrentLinkedQueue[ReceivedBlockTrackerLogEvent]() + protected val walWriteQueue = new LinkedBlockingQueue[ReceivedBlockTrackerLogEvent]() // stores the status for wal writes added to the queue. Exposed for tests protected val walWriteStatusMap = - new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, WALWriteStatus]() + new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, Promise[Boolean]]() - private val WAL_WRITE_STATUS_CHECK_BACKOFF = 10 // 10 millis private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds private val writeAheadLogBatchWriter: Option[BatchLogWriter] = createBatchWriteAheadLogWriter() + private val batchWriterThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batch-writer-thead-pool")) private var lastAllocatedBatchTime: Time = null @@ -96,12 +100,10 @@ private[streaming] class ReceivedBlockTracker( } /** Add received block. This event will get written to the write ahead log (if enabled). */ - private def addBlock0( - receivedBlockInfo: ReceivedBlockInfo, - writeOp: ReceivedBlockTrackerLogEvent => Boolean): Boolean = { + def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { try { - val result = writeOp(BlockAdditionEvent(receivedBlockInfo)) - if (result) { + val writeResult = writeToLog(BlockAdditionEvent(receivedBlockInfo)) + if (writeResult) { afterBlockAddAcknowledged(receivedBlockInfo) logDebug(s"Stream ${receivedBlockInfo.streamId} received " + s"block ${receivedBlockInfo.blockStoreResult.blockId}") @@ -109,9 +111,9 @@ private[streaming] class ReceivedBlockTracker( logDebug(s"Failed to acknowledge stream ${receivedBlockInfo.streamId} receiving " + s"block ${receivedBlockInfo.blockStoreResult.blockId} in the Write Ahead Log.") } - result + writeResult } catch { - case e: Exception => + case NonFatal(e) => logError(s"Error adding block $receivedBlockInfo", e) false } @@ -122,29 +124,17 @@ private[streaming] class ReceivedBlockTracker( getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo } - /** Add received block. This event will get written to the write ahead log (if enabled). */ - def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - addBlock0(receivedBlockInfo, writeToLog) - } - - def addBlockAsync(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - addBlock0(receivedBlockInfo, writeToLogAsync) - } - /** * Allocate all unallocated blocks to the given batch. * This event will get written to the write ahead log (if enabled). */ - private def allocateBlocksToBatch0( - batchTime: Time, - writeOp: ReceivedBlockTrackerLogEvent => Boolean): Unit = { + def allocateBlocksToBatch(batchTime: Time): Unit = { if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { val streamIdToBlocks = streamIds.map { streamId => (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) - val result = writeOp(BatchAllocationEvent(batchTime, allocatedBlocks)) - if (result) { + if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { afterBatchAllocationAcknowledged(batchTime, allocatedBlocks) } else { logInfo(s"Possibly processed batch $batchTime need to be processed again in WAL recovery") @@ -161,18 +151,6 @@ private[streaming] class ReceivedBlockTracker( } } - /** - * Allocate all unallocated blocks to the given batch. - * This event will get written to the write ahead log (if enabled). - */ - def allocateBlocksToBatch(batchTime: Time): Unit = { - allocateBlocksToBatch0(batchTime, writeToLog) - } - - def allocateBlocksToBatchAsync(batchTime: Time): Unit = { - allocateBlocksToBatch0(batchTime, writeToLogAsync) - } - /** Update in-memory state after batch allocation event has been logged to WAL. */ private def afterBatchAllocationAcknowledged( batchTime: Time, @@ -212,50 +190,23 @@ private[streaming] class ReceivedBlockTracker( * Clean up block information of old batches. If waitForCompletion is true, this method * returns only after the files are cleaned up. */ - private def cleanupOldBatches0( - cleanupThreshTime: Time, - waitForCompletion: Boolean, - writeOp: ReceivedBlockTrackerLogEvent => Boolean): Unit = { + def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { require(cleanupThreshTime.milliseconds < clock.getTimeMillis()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq logInfo("Deleting batches " + timesToCleanup) - if (writeOp(BatchCleanupEvent(timesToCleanup))) { - afterBatchCleanupAcknowledged(cleanupThreshTime, waitForCompletion, timesToCleanup) + if (writeToLog(BatchCleanupEvent(timesToCleanup))) { + timeToAllocatedBlocks --= timesToCleanup + writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) } else { logWarning("Failed to acknowledge batch clean up in the Write Ahead Log.") } } - /** - * Clean up block information of old batches. If waitForCompletion is true, this method - * returns only after the files are cleaned up. - */ - def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = { - cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLog) - } - - /** - * Clean up block information of old batches. If waitForCompletion is true, this method - * returns only after the files are cleaned up. Not really async, the event will be written to - * a WAL within a batch, if WAL's are enabled. - */ - def cleanupOldBatchesAsync(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = { - cleanupOldBatches0(cleanupThreshTime, waitForCompletion, writeToLogAsync) - } - - /** Update in-memory state after clean up event has been logged to WAL. */ - private def afterBatchCleanupAcknowledged( - cleanupThreshTime: Time, - waitForCompletion: Boolean, - timesToCleanup: Seq[Time]): Unit = synchronized { - timeToAllocatedBlocks --= timesToCleanup - writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion)) - } - /** Stop the block tracker. */ def stop() { writeAheadLogBatchWriter.foreach { _.stop() } writeAheadLogOption.foreach { _.close() } + batchWriterThreadPool.shutdownNow() } /** @@ -287,7 +238,8 @@ private[streaming] class ReceivedBlockTracker( def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { event match { - case CombinedRBTLogEvent(events) => events.foreach(resolveEvent) + case CombinedReceivedBlockTrackerLogEvent(events) => + events.foreach(resolveEvent) case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => @@ -308,33 +260,21 @@ private[streaming] class ReceivedBlockTracker( } /** Write an update to the tracker to the write ahead log */ - private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { - if (isWriteAheadLogEnabled) { + private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { + if (!isWriteAheadLogEnabled) return true + if (WriteAheadLogUtils.isBatchingEnabled(conf)) { + val promise = Promise[Boolean]() + walWriteStatusMap.put(record, promise) + walWriteQueue.offer(record) + Await.result(promise.future.recover { case _ => false }(batchWriterThreadPool), + WAL_WRITE_STATUS_TIMEOUT.milliseconds) + } else { writeAheadLogOption.foreach { logManager => logTrace(s"Writing record: $record") logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) } + true } - true - } - - import WALWriteStatus._ - - /** - * Adds LogEvents to a queue so that they can be batched and written to the WAL. - * Exposed for testing. - */ - private[streaming] def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = { - if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled - walWriteStatusMap.put(event, Pending) - walWriteQueue.offer(event) - var timedOut = false - val start = clock.getTimeMillis() - while (walWriteStatusMap.get(event) == Pending) { - Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) - if (clock.getTimeMillis() - WAL_WRITE_STATUS_TIMEOUT > start) timedOut = true - } - walWriteStatusMap.remove(event) == Success } /** Get the queue of received blocks belonging to a particular stream */ @@ -358,10 +298,9 @@ private[streaming] class ReceivedBlockTracker( if (!WriteAheadLogUtils.isBatchingEnabled(conf)) return None val writer = checkpointDirOption.map(_ => new BatchLogWriter) writer.foreach { runnable => - new Thread(runnable, "Batch WAL Writer").start() - Runtime.getRuntime.addShutdownHook(new Thread { - override def run(): Unit = runnable.stop() - }) + val thread = new Thread(runnable, "Batch WAL Writer") + thread.setDaemon(true) + thread.start() } writer } @@ -374,41 +313,60 @@ private[streaming] class ReceivedBlockTracker( var active: Boolean = true - private def writeRecords(records: List[ReceivedBlockTrackerLogEvent]): Unit = { + private def writeRecords(records: Seq[ReceivedBlockTrackerLogEvent]): Unit = { writeAheadLogOption.foreach { logManager => if (records.nonEmpty) { logDebug(s"Batched ${records.length} records for WAL write") - logManager.write(ByteBuffer.wrap(Utils.serialize(CombinedRBTLogEvent(records))), - clock.getTimeMillis()) + logManager.write(ByteBuffer.wrap(Utils.serialize( + CombinedReceivedBlockTrackerLogEvent(records))), clock.getTimeMillis()) } } } def stop(): Unit = { + logInfo("Stopping Batch Write Ahead Log writer.") active = false } private def flushRecords(): Unit = { val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]() - while (!walWriteQueue.isEmpty) { - buffer.append(walWriteQueue.poll()) + try { + buffer.append(walWriteQueue.take()) + while (!walWriteQueue.isEmpty) { + buffer.append(walWriteQueue.poll()) + } + } catch { + case _: InterruptedException => + logWarning("Batch Write Ahead Log Writer queue interrupted.") + } + def updateRecordStatus(record: ReceivedBlockTrackerLogEvent, successful: Boolean): Unit = { + val promise = walWriteStatusMap.get(record) + if (promise == null) { + logError(s"Promise for writing record $record not found in status map!") + } else { + promise.success(successful) + } } - val records = buffer.toList try { - writeRecords(records) - records.foreach(walWriteStatusMap.put(_, Success)) + writeRecords(buffer) + buffer.foreach(updateRecordStatus(_, successful = true)) } catch { - case e: Exception => - logWarning(s"Batch WAL Writer failed to write $records") - records.foreach(walWriteStatusMap.put(_, Fail)) + case NonFatal(e) => + logWarning(s"Batch WAL Writer failed to write $buffer", e) + buffer.foreach(updateRecordStatus(_, successful = false)) } } override def run(): Unit = { while (active) { - flushRecords() - Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF) + try { + flushRecords() + } catch { + case NonFatal(e) => + logError("Exception while flushing records in Batch Write Ahead Log writer.", e) + } } + logInfo("Batch Write Ahead Log writer shutting down.") } } } @@ -418,11 +376,3 @@ private[streaming] object ReceivedBlockTracker { new Path(checkpointDir, "receivedBlockMetadata").toString } } - -/** State of LogEvents during batching. */ -private[streaming] trait WALWriteStatus -private[streaming] object WALWriteStatus { - private[streaming] case object Pending extends WALWriteStatus - private[streaming] case object Success extends WALWriteStatus - private[streaming] case object Fail extends WALWriteStatus -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 39f413504f33c..8e729183a9257 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -200,11 +200,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Allocate all unallocated blocks to the given batch. */ def allocateBlocksToBatch(batchTime: Time): Unit = { if (receiverInputStreams.nonEmpty) { - if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { - receivedBlockTracker.allocateBlocksToBatchAsync(batchTime) - } else { - receivedBlockTracker.allocateBlocksToBatch(batchTime) - } + receivedBlockTracker.allocateBlocksToBatch(batchTime) } } @@ -224,11 +220,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false */ def cleanupOldBlocksAndBatches(cleanupThreshTime: Time) { // Clean up old block and batch metadata - if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { - receivedBlockTracker.cleanupOldBatchesAsync(cleanupThreshTime, waitForCompletion = false) - } else { - receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) - } + receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) // Signal the receivers to delete old block data if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) { @@ -329,11 +321,6 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false receivedBlockTracker.addBlock(receivedBlockInfo) } - /** Add new blocks for the given stream */ - private def addBlockAsync(receivedBlockInfo: ReceivedBlockInfo): Boolean = { - receivedBlockTracker.addBlockAsync(receivedBlockInfo) - } - /** Report error sent by a receiver */ private def reportError(streamId: Int, message: String, error: String) { val newReceiverTrackingInfo = receiverTrackingInfos.get(streamId) match { @@ -505,7 +492,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(successful) case AddBlock(receivedBlockInfo) => if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { - val f = Future(addBlockAsync(receivedBlockInfo))(walBatchingThreadPool) + val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool) f.onComplete(result => context.reply(result.get))(walBatchingThreadPool) } else { context.reply(addBlock(receivedBlockInfo)) @@ -620,6 +607,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def onStop(): Unit = { submitJobThreadPool.shutdownNow() + walBatchingThreadPool.shutdownNow() } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 3b82b26cb54fd..75483adfbb757 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -38,7 +38,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { val DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY = "spark.streaming.driver.writeAheadLog.rollingIntervalSecs" val DRIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.driver.writeAheadLog.maxFailures" - val DRIVER_WAL_BATCHING_CONF_KEY = "spark.streaming.driver.writeAheadLog.enableBatching" + val DRIVER_WAL_BATCHING_CONF_KEY = "spark.streaming.driver.writeAheadLog.allowBatching" val DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY = "spark.streaming.driver.writeAheadLog.closeFileAfterWrite" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 1d6effd6b3391..be591e500c301 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -55,7 +55,7 @@ abstract class ReceivedBlockTrackerSuite conf = new SparkConf().setMaster("local[2]").setAppName("ReceivedBlockTrackerSuite") checkpointDirectory = Utils.createTempDir() if (isBatchingEnabled) { - conf.set("spark.streaming.driver.writeAheadLog.enableBatching", "true") + conf.set("spark.streaming.driver.writeAheadLog.allowBatching", "true") } } @@ -68,30 +68,10 @@ abstract class ReceivedBlockTrackerSuite def addBlockInfos( tracker: ReceivedBlockTracker, blockInfos: Seq[ReceivedBlockInfo] = generateBlockInfos()): Seq[ReceivedBlockInfo] = { - if (isBatchingEnabled) { - blockInfos.map(tracker.addBlockAsync) - } else { - blockInfos.map(tracker.addBlock) - } + blockInfos.map(tracker.addBlock) blockInfos } - def allocateBlocksToBatch(t: Time, tracker: ReceivedBlockTracker): Unit = { - if (isBatchingEnabled) { - tracker.allocateBlocksToBatchAsync(t) - } else { - tracker.allocateBlocksToBatch(t) - } - } - - def cleanupOldBatches(t: Time, wait: Boolean, tracker: ReceivedBlockTracker): Unit = { - if (isBatchingEnabled) { - tracker.cleanupOldBatchesAsync(t, wait) - } else { - tracker.cleanupOldBatches(t, wait) - } - } - test("block addition, and block to batch allocation") { val receivedBlockTracker = createTracker(setCheckpointDir = false) receivedBlockTracker.isWriteAheadLogEnabled should be (false) // should be disable by default @@ -104,24 +84,24 @@ abstract class ReceivedBlockTrackerSuite receivedBlockTracker.hasUnallocatedReceivedBlocks should be (true) // Allocate the blocks to a batch and verify that all of them have been allocated - allocateBlocksToBatch(1, receivedBlockTracker) + receivedBlockTracker.allocateBlocksToBatch(1) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos receivedBlockTracker.getBlocksOfBatch(1) shouldEqual Map(streamId -> blockInfos) receivedBlockTracker.getUnallocatedBlocks(streamId) shouldBe empty receivedBlockTracker.hasUnallocatedReceivedBlocks should be (false) // Allocate no blocks to another batch - allocateBlocksToBatch(2, receivedBlockTracker) + receivedBlockTracker.allocateBlocksToBatch(2) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty receivedBlockTracker.getBlocksOfBatch(2) shouldEqual Map(streamId -> Seq.empty) // Verify that older batches have no operation on batch allocation, // will return the same blocks as previously allocated. - allocateBlocksToBatch(1, receivedBlockTracker) + receivedBlockTracker.allocateBlocksToBatch(1) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos addBlockInfos(receivedBlockTracker, blockInfos) - allocateBlocksToBatch(2, receivedBlockTracker) + receivedBlockTracker.allocateBlocksToBatch(2) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos } @@ -177,7 +157,7 @@ abstract class ReceivedBlockTrackerSuite // Allocate blocks to batch and verify whether the unallocated blocks got allocated val batchTime1 = manualClock.getTimeMillis() - allocateBlocksToBatch(batchTime1, tracker2) + tracker2.allocateBlocksToBatch(batchTime1) tracker2.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 tracker2.getBlocksOfBatch(batchTime1) shouldEqual Map(streamId -> blockInfos1) @@ -185,7 +165,7 @@ abstract class ReceivedBlockTrackerSuite incrementTime() val batchTime2 = manualClock.getTimeMillis() val blockInfos2 = addBlockInfos(tracker2) - allocateBlocksToBatch(batchTime2, tracker2) + tracker2.allocateBlocksToBatch(batchTime2) tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 // Verify whether log has correct contents val expectedWrittenData2 = expectedWrittenData1 ++ @@ -204,7 +184,7 @@ abstract class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - cleanupOldBatches(batchTime2, true, tracker3) + tracker3.cleanupOldBatches(batchTime2, true) // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty getWrittenLogData(getWriteAheadLogFiles().last) should contain(createBatchCleanup(batchTime1)) @@ -266,7 +246,7 @@ abstract class ReceivedBlockTrackerSuite file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq }.flatMap { byteBuffer => Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { - case CombinedRBTLogEvent(events) => events + case CombinedReceivedBlockTrackerLogEvent(events) => events case others: ReceivedBlockTrackerLogEvent => Seq(others) } }.toList @@ -312,15 +292,16 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock override def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = None - def updateWALWriteStatus(event: ReceivedBlockTrackerLogEvent, status: WALWriteStatus): Unit = { - walWriteStatusMap.put(event, status) + def updateWALWriteStatus(event: ReceivedBlockTrackerLogEvent, successful: Boolean): Unit = { + val promise = walWriteStatusMap.get(event) + promise.success(successful) walWriteQueue.poll() } def getQueueLength(): Int = walWriteQueue.size() def addToQueue(event: ReceivedBlockTrackerLogEvent): Boolean = { - writeToLogAsync(event) + writeToLog(event) } } @@ -335,8 +316,6 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock result } - import WALWriteStatus._ - test("records get added to a queue") { val numSuccess = new AtomicInteger() val numFail = new AtomicInteger() @@ -350,7 +329,7 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock def eventFuture(event: ReceivedBlockTrackerLogEvent): Unit = { val f = Future(rbt.addToQueue(event))(walBatchingThreadPool) - f.onSuccess{ case v => + f.onSuccess { case v => if (v) numSuccess.incrementAndGet() else numFail.incrementAndGet() }(walBatchingThreadPool) } @@ -367,7 +346,7 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock assert(numSuccess.get() === 0) assert(numFail.get() === 0) - rbt.updateWALWriteStatus(event1, Fail) + rbt.updateWALWriteStatus(event1, successful = false) assert(waitUntilTrue(getNumFail, 1)) assert(waitUntilTrue(rbt.getQueueLength, 0)) @@ -375,15 +354,15 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock eventFuture(event3) eventFuture(event4) assert(waitUntilTrue(rbt.getQueueLength, 3)) - rbt.updateWALWriteStatus(event2, Success) - rbt.updateWALWriteStatus(event3, Success) + rbt.updateWALWriteStatus(event2, successful = true) + rbt.updateWALWriteStatus(event3, successful = true) assert(waitUntilTrue(getNumSuccess, 2)) assert(waitUntilTrue(rbt.getQueueLength, 1)) eventFuture(event5) assert(waitUntilTrue(rbt.getQueueLength, 2)) - rbt.updateWALWriteStatus(event4, Success) - rbt.updateWALWriteStatus(event5, Success) + rbt.updateWALWriteStatus(event4, successful = true) + rbt.updateWALWriteStatus(event5, successful = true) assert(waitUntilTrue(getNumSuccess, 4)) assert(waitUntilTrue(rbt.getQueueLength, 0)) } From 48c9f2244d0ab21f53ab28b6cd77ed8238d397d5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 28 Oct 2015 16:09:03 -0700 Subject: [PATCH 06/26] minor improvements --- .../scheduler/ReceivedBlockTracker.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 78ee331c29d16..03d86215a9e26 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -18,8 +18,9 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} +import java.util.concurrent.{TimeoutException, ConcurrentHashMap, LinkedBlockingQueue} +import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -266,8 +267,15 @@ private[streaming] class ReceivedBlockTracker( val promise = Promise[Boolean]() walWriteStatusMap.put(record, promise) walWriteQueue.offer(record) - Await.result(promise.future.recover { case _ => false }(batchWriterThreadPool), - WAL_WRITE_STATUS_TIMEOUT.milliseconds) + try { + Await.result(promise.future.recover { case _ => false}(batchWriterThreadPool), + WAL_WRITE_STATUS_TIMEOUT.milliseconds) + } catch { + case e: TimeoutException => + logWarning(s"Write to Write Ahead Log promise timed out after $WAL_WRITE_STATUS_TIMEOUT " + + s"millis for record: $record") + false + } } else { writeAheadLogOption.foreach { logManager => logTrace(s"Writing record: $record") @@ -332,9 +340,7 @@ private[streaming] class ReceivedBlockTracker( val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]() try { buffer.append(walWriteQueue.take()) - while (!walWriteQueue.isEmpty) { - buffer.append(walWriteQueue.poll()) - } + walWriteQueue.drainTo(buffer) } catch { case _: InterruptedException => logWarning("Batch Write Ahead Log Writer queue interrupted.") From 596cf8f841472aa07bfee5bca6508dfd9e4f14fc Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 28 Oct 2015 16:27:36 -0700 Subject: [PATCH 07/26] remove implicit --- .../spark/streaming/scheduler/ReceivedBlockTracker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 03d86215a9e26..1d8740d13789f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer import java.util.concurrent.{TimeoutException, ConcurrentHashMap, LinkedBlockingQueue} -import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -340,7 +339,8 @@ private[streaming] class ReceivedBlockTracker( val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]() try { buffer.append(walWriteQueue.take()) - walWriteQueue.drainTo(buffer) + val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 + logDebug(s"Received $numBatched records from queue") } catch { case _: InterruptedException => logWarning("Batch Write Ahead Log Writer queue interrupted.") From 13cab44bcd1ed4f5c1fa5bb855375f137fa15768 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 28 Oct 2015 16:47:56 -0700 Subject: [PATCH 08/26] fix scalastyle --- .../spark/streaming/scheduler/ReceivedBlockTracker.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 1d8740d13789f..1d2443f8fcfd3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -271,8 +271,8 @@ private[streaming] class ReceivedBlockTracker( WAL_WRITE_STATUS_TIMEOUT.milliseconds) } catch { case e: TimeoutException => - logWarning(s"Write to Write Ahead Log promise timed out after $WAL_WRITE_STATUS_TIMEOUT " + - s"millis for record: $record") + logWarning(s"Write to Write Ahead Log promise timed out after " + + s"$WAL_WRITE_STATUS_TIMEOUT millis for record: $record") false } } else { From ea62d253f7e8d33484ab19263a7b61968b774d67 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 29 Oct 2015 11:19:56 -0700 Subject: [PATCH 09/26] some updates for testing --- .../scheduler/ReceivedBlockTracker.scala | 123 +--------- .../streaming/util/BatchedWriteAheadLog.scala | 163 ++++++++++++ .../streaming/util/WriteAheadLogUtils.scala | 7 +- .../streaming/ReceivedBlockTrackerSuite.scala | 23 +- .../streaming/util/WriteAheadLogSuite.scala | 232 ++++++++++-------- 5 files changed, 314 insertions(+), 234 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 1d2443f8fcfd3..4800d801fd2c8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -18,13 +18,9 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer -import java.util.concurrent.{TimeoutException, ConcurrentHashMap, LinkedBlockingQueue} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Await, Promise} -import scala.concurrent.duration._ import scala.language.implicitConversions import scala.util.control.NonFatal @@ -33,7 +29,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.streaming.Time import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} -import org.apache.spark.util.{ThreadUtils, Clock, Utils} +import org.apache.spark.util.{Clock, Utils} import org.apache.spark.{Logging, SparkConf} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ @@ -45,8 +41,9 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: extends ReceivedBlockTrackerLogEvent private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent +// Each event is in serialized format inside the array private[streaming] case class CombinedReceivedBlockTrackerLogEvent( - events: Seq[ReceivedBlockTrackerLogEvent]) extends ReceivedBlockTrackerLogEvent + events: Array[Array[Byte]]) extends ReceivedBlockTrackerLogEvent /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] @@ -79,18 +76,6 @@ private[streaming] class ReceivedBlockTracker( private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue] private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] private val writeAheadLogOption = createWriteAheadLog() - // exposed for tests - protected val walWriteQueue = new LinkedBlockingQueue[ReceivedBlockTrackerLogEvent]() - - // stores the status for wal writes added to the queue. Exposed for tests - protected val walWriteStatusMap = - new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, Promise[Boolean]]() - - private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds - - private val writeAheadLogBatchWriter: Option[BatchLogWriter] = createBatchWriteAheadLogWriter() - private val batchWriterThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("wal-batch-writer-thead-pool")) private var lastAllocatedBatchTime: Time = null @@ -204,9 +189,7 @@ private[streaming] class ReceivedBlockTracker( /** Stop the block tracker. */ def stop() { - writeAheadLogBatchWriter.foreach { _.stop() } writeAheadLogOption.foreach { _.close() } - batchWriterThreadPool.shutdownNow() } /** @@ -239,7 +222,7 @@ private[streaming] class ReceivedBlockTracker( def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { event match { case CombinedReceivedBlockTrackerLogEvent(events) => - events.foreach(resolveEvent) + events.foreach(r => resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent](r))) case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => @@ -262,25 +245,9 @@ private[streaming] class ReceivedBlockTracker( /** Write an update to the tracker to the write ahead log */ private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (!isWriteAheadLogEnabled) return true - if (WriteAheadLogUtils.isBatchingEnabled(conf)) { - val promise = Promise[Boolean]() - walWriteStatusMap.put(record, promise) - walWriteQueue.offer(record) - try { - Await.result(promise.future.recover { case _ => false}(batchWriterThreadPool), - WAL_WRITE_STATUS_TIMEOUT.milliseconds) - } catch { - case e: TimeoutException => - logWarning(s"Write to Write Ahead Log promise timed out after " + - s"$WAL_WRITE_STATUS_TIMEOUT millis for record: $record") - false - } - } else { - writeAheadLogOption.foreach { logManager => - logTrace(s"Writing record: $record") - logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) - } - true + writeAheadLogOption.exists { logManager => + logTrace(s"Writing record: $record") + logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) != null } } @@ -297,84 +264,8 @@ private[streaming] class ReceivedBlockTracker( } } - /** - * Creates a WAL Writer in a separate thread to enable batching of log events. - * Exposed for tests. - */ - protected def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = { - if (!WriteAheadLogUtils.isBatchingEnabled(conf)) return None - val writer = checkpointDirOption.map(_ => new BatchLogWriter) - writer.foreach { runnable => - val thread = new Thread(runnable, "Batch WAL Writer") - thread.setDaemon(true) - thread.start() - } - writer - } - /** Check if the write ahead log is enabled. This is only used for testing purposes. */ private[streaming] def isWriteAheadLogEnabled: Boolean = writeAheadLogOption.nonEmpty - - /** A helper class that writes LogEvents in a separate thread to allow for batching. */ - private[streaming] class BatchLogWriter extends Runnable { - - var active: Boolean = true - - private def writeRecords(records: Seq[ReceivedBlockTrackerLogEvent]): Unit = { - writeAheadLogOption.foreach { logManager => - if (records.nonEmpty) { - logDebug(s"Batched ${records.length} records for WAL write") - logManager.write(ByteBuffer.wrap(Utils.serialize( - CombinedReceivedBlockTrackerLogEvent(records))), clock.getTimeMillis()) - } - } - } - - def stop(): Unit = { - logInfo("Stopping Batch Write Ahead Log writer.") - active = false - } - - private def flushRecords(): Unit = { - val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]() - try { - buffer.append(walWriteQueue.take()) - val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 - logDebug(s"Received $numBatched records from queue") - } catch { - case _: InterruptedException => - logWarning("Batch Write Ahead Log Writer queue interrupted.") - } - def updateRecordStatus(record: ReceivedBlockTrackerLogEvent, successful: Boolean): Unit = { - val promise = walWriteStatusMap.get(record) - if (promise == null) { - logError(s"Promise for writing record $record not found in status map!") - } else { - promise.success(successful) - } - } - try { - writeRecords(buffer) - buffer.foreach(updateRecordStatus(_, successful = true)) - } catch { - case NonFatal(e) => - logWarning(s"Batch WAL Writer failed to write $buffer", e) - buffer.foreach(updateRecordStatus(_, successful = false)) - } - } - - override def run(): Unit = { - while (active) { - try { - flushRecords() - } catch { - case NonFatal(e) => - logError("Exception while flushing records in Batch Write Ahead Log writer.", e) - } - } - logInfo("Batch Write Ahead Log writer shutting down.") - } - } } private[streaming] object ReceivedBlockTracker { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala new file mode 100644 index 0000000000000..8c46b8f7ff022 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -0,0 +1,163 @@ +package org.apache.spark.streaming.util + +import java.nio.ByteBuffer +import java.util.concurrent.{LinkedBlockingQueue, TimeoutException} +import java.util.{Iterator => JIterator} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{Await, ExecutionContext, Promise} +import scala.concurrent.duration._ +import scala.util.control.NonFatal + +import org.apache.spark.Logging +import org.apache.spark.streaming.scheduler.CombinedReceivedBlockTrackerLogEvent +import org.apache.spark.util.{Utils, ThreadUtils} + +/** + * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled with + * the timestamp for the write request of the record, and the promise that will block the write + * request, while a separate thread is actually performing the write. + */ +private[util] case class RecordBuffer( + record: ByteBuffer, + time: Long, + promise: Promise[WriteAheadLogRecordHandle]) + +/** + * A wrapper for a WriteAheadLog that batches records before writing data. All other methods will + * be passed on to the wrapped class. + */ +private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) + extends WriteAheadLog with Logging { + + /** A thread pool for fulfilling log write promises */ + private val batchWriterThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batch-writer-thead-pool")) + + // exposed for tests + protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]() + + private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds + + private val writeAheadLogBatchWriter: BatchedLogWriter = startBatchedWriterThread() + + /** + * Write a byte buffer to the log file. This method adds the byteBuffer to a queue and blocks + * until the record is properly written by the parent. + */ + override def write(byteBuffer: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { + val promise = Promise[WriteAheadLogRecordHandle]() + walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise)) + try { + Await.result(promise.future.recover { case _ => null }(batchWriterThreadPool), + WAL_WRITE_STATUS_TIMEOUT.milliseconds) + } catch { + case e: TimeoutException => + logWarning(s"Write to Write Ahead Log promise timed out after " + + s"$WAL_WRITE_STATUS_TIMEOUT millis for record.") + null + } + } + + /** + * Read a segment from an existing Write Ahead Log. + * + * This method is handled by the parent WriteAheadLog. + */ + override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = { + parent.read(segment) + } + + /** + * Read all the existing logs from the log directory. + * + * This method is handled by the parent WriteAheadLog. + */ + override def readAll(): JIterator[ByteBuffer] = { + parent.readAll() + } + + /** + * Delete the log files that are older than the threshold time. + * + * This method is handled by the parent WriteAheadLog. + */ + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { + parent.clean(threshTime, waitForCompletion) + } + + + /** + * Stop the manager, close any open log writer. + * + * This method is handled by the parent WriteAheadLog. + */ + override def close(): Unit = { + writeAheadLogBatchWriter.stop() + batchWriterThreadPool.shutdownNow() + parent.close() + } + + /** Start the actual log writer on a separate thread. */ + private def startBatchedWriterThread(): BatchedLogWriter = { + val writer = new BatchedLogWriter() + val thread = new Thread(writer, "Batched WAL Writer") + thread.setDaemon(true) + thread.start() + writer + } + + /** A helper class that writes LogEvents in a separate thread to allow for batching. */ + private[util] class BatchedLogWriter extends Runnable { + + private var active: Boolean = true + private val buffer = new ArrayBuffer[RecordBuffer]() + + override def run(): Unit = { + while (active) { + try { + flushRecords() + } catch { + case NonFatal(e) => + logError("Exception while flushing records in Batch Write Ahead Log writer.", e) + } + } + logInfo("Batch Write Ahead Log writer shutting down.") + } + + def stop(): Unit = { + logInfo("Stopping Batch Write Ahead Log writer.") + active = false + } + + /** Write all the records in the buffer to the write ahead log. */ + private def flushRecords(): Unit = { + try { + buffer.append(walWriteQueue.take()) + val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 + logDebug(s"Received $numBatched records from queue") + } catch { + case _: InterruptedException => + logWarning("Batch Write Ahead Log Writer queue interrupted.") + } + try { + var segment: WriteAheadLogRecordHandle = null + if (buffer.length > 0) { + logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") + // we take the latest record for the time to ensure that we don't clean up files earlier + // than the expiration date of the records + val time = buffer.last.time + segment = parent.write(ByteBuffer.wrap(Utils.serialize( + CombinedReceivedBlockTrackerLogEvent(buffer.map(_.record.array()).toArray))), time) + } + buffer.foreach(_.promise.success(segment)) + } catch { + case NonFatal(e) => + logWarning(s"Batch WAL Writer failed to write $buffer", e) + buffer.foreach(_.promise.success(null)) + } + buffer.clear() + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 75483adfbb757..ba93019fa9dca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -120,7 +120,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { } else { sparkConf.getOption(RECEIVER_WAL_CLASS_CONF_KEY) } - classNameOption.map { className => + val wal = classNameOption.map { className => try { instantiateClass( Utils.classForName(className).asInstanceOf[Class[_ <: WriteAheadLog]], sparkConf) @@ -133,6 +133,11 @@ private[streaming] object WriteAheadLogUtils extends Logging { getRollingIntervalSecs(sparkConf, isDriver), getMaxFailures(sparkConf, isDriver), shouldCloseFileAfterWrite(sparkConf, isDriver)) } + if (isDriver && isBatchingEnabled(sparkConf)) { + new BatchedWriteAheadLog(wal) + } else { + wal + } } /** Instantiate the class, either using single arg constructor or zero arg constructor */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index be591e500c301..69d103443b4bb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadL import org.apache.spark.streaming.util.WriteAheadLogSuite._ import org.apache.spark.util._ -abstract class ReceivedBlockTrackerSuite +class ReceivedBlockTrackerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { val hadoopConf = new Configuration() @@ -49,14 +49,10 @@ abstract class ReceivedBlockTrackerSuite var allReceivedBlockTrackers = new ArrayBuffer[ReceivedBlockTracker]() var checkpointDirectory: File = null var conf: SparkConf = null - val isBatchingEnabled: Boolean before { conf = new SparkConf().setMaster("local[2]").setAppName("ReceivedBlockTrackerSuite") checkpointDirectory = Utils.createTempDir() - if (isBatchingEnabled) { - conf.set("spark.streaming.driver.writeAheadLog.allowBatching", "true") - } } after { @@ -244,11 +240,8 @@ abstract class ReceivedBlockTrackerSuite : Seq[ReceivedBlockTrackerLogEvent] = { logFiles.flatMap { file => new FileBasedWriteAheadLogReader(file, hadoopConf).toSeq - }.flatMap { byteBuffer => - Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { - case CombinedReceivedBlockTrackerLogEvent(events) => events - case others: ReceivedBlockTrackerLogEvent => Seq(others) - } + }.map { byteBuffer => + Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) }.toList } @@ -275,14 +268,11 @@ abstract class ReceivedBlockTrackerSuite implicit def timeToMillis(time: Time): Long = time.milliseconds } -class SyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite { - override val isBatchingEnabled = false -} - +/** class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with MockitoSugar { override val isBatchingEnabled = true - /** Class that will help us test batching. */ + private class TestRBT( clock: Clock = new SystemClock, cpDirOption: Option[String] = Some(checkpointDirectory.toString), @@ -304,6 +294,7 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock writeToLog(event) } } + Class that will help us test batching. private def waitUntilTrue(f: () => Int, value: Int): Boolean = { val timeOut = 2000 @@ -366,4 +357,6 @@ class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with Mock assert(waitUntilTrue(getNumSuccess, 4)) assert(waitUntilTrue(rbt.getQueueLength, 0)) } + } + */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 93ae41a3d2ecd..ff22dcbf5057e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -34,7 +34,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { +abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfter { import WriteAheadLogSuite._ @@ -42,7 +42,9 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { var tempDir: File = null var testDir: String = null var testFile: String = null - var writeAheadLog: FileBasedWriteAheadLog = null + var writeAheadLog: WriteAheadLog = null + protected val allowBatching: Boolean + protected val closeFileAfterWrite: Boolean before { tempDir = Utils.createTempDir() @@ -102,6 +104,114 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { } } + test("WriteAheadLog - read rotating logs") { + // Write data manually for testing reading through WriteAheadLog + val writtenData = (1 to 10).map { i => + val data = generateRandomData() + val file = testDir + s"/log-$i-$i" + writeDataManually(data, file) + data + }.flatten + + val logDirectoryPath = new Path(testDir) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + assert(fileSystem.exists(logDirectoryPath) === true) + + // Read data using manager and verify + val readData = readDataUsingWriteAheadLog(testDir) + assert(readData === writtenData) + } + + test("WriteAheadLog - recover past logs when creating new manager") { + // Write data with manager, recover with new manager and verify + val dataToWrite = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val readData = readDataUsingWriteAheadLog(testDir) + assert(dataToWrite === readData) + } + + test("WriteAheadLog - clean old logs") { + logCleanUpTest(waitForCompletion = false) + } + + test("WriteAheadLog - clean old logs synchronously") { + logCleanUpTest(waitForCompletion = true) + } + + private def logCleanUpTest(waitForCompletion: Boolean): Unit = { + // Write data with manager, recover with new manager and verify + val manualClock = new ManualClock + val dataToWrite = generateRandomData() + writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + + writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion) + + if (waitForCompletion) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } else { + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } + } + } + + test("WriteAheadLog - handling file errors while reading rotating logs") { + // Generate a set of log files + val manualClock = new ManualClock + val dataToWrite1 = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock) + val logFiles1 = getLogFilesInDirectory(testDir) + assert(logFiles1.size > 1) + + + // Recover old files and generate a second set of log files + val dataToWrite2 = generateRandomData() + manualClock.advance(100000) + writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock) + val logFiles2 = getLogFilesInDirectory(testDir) + assert(logFiles2.size > logFiles1.size) + + // Read the files and verify that all the written data can be read + val readData1 = readDataUsingWriteAheadLog(testDir) + assert(readData1 === (dataToWrite1 ++ dataToWrite2)) + + // Corrupt the first set of files so that they are basically unreadable + logFiles1.foreach { f => + val raf = new FileOutputStream(f, true).getChannel() + raf.truncate(1) + raf.close() + } + + // Verify that the corrupted files do not prevent reading of the second set of data + val readData = readDataUsingWriteAheadLog(testDir) + assert(readData === dataToWrite2) + } + + test("WriteAheadLog - do not create directories or files unless write") { + val nonexistentTempPath = File.createTempFile("test", "") + nonexistentTempPath.delete() + assert(!nonexistentTempPath.exists()) + + val writtenSegment = writeDataManually(generateRandomData(), testFile) + val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath, + new Configuration(), 1, 1, closeFileAfterWrite = false) + assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") + wal.read(writtenSegment.head) + assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") + } +} + +class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { + + import WriteAheadLogSuite._ + + override protected val closeFileAfterWrite: Boolean = false + override protected val allowBatching: Boolean = false + test("FileBasedWriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) @@ -217,106 +327,22 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter { val writtenData = logFiles.flatMap { file => readDataManually(file)} assert(writtenData === dataToWrite) } +} - test("FileBasedWriteAheadLog - read rotating logs") { - // Write data manually for testing reading through WriteAheadLog - val writtenData = (1 to 10).map { i => - val data = generateRandomData() - val file = testDir + s"/log-$i-$i" - writeDataManually(data, file) - data - }.flatten - - val logDirectoryPath = new Path(testDir) - val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - assert(fileSystem.exists(logDirectoryPath) === true) - - // Read data using manager and verify - val readData = readDataUsingWriteAheadLog(testDir) - assert(readData === writtenData) - } - - test("FileBasedWriteAheadLog - recover past logs when creating new manager") { - // Write data with manager, recover with new manager and verify - val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite) - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - val readData = readDataUsingWriteAheadLog(testDir) - assert(dataToWrite === readData) - } - - test("FileBasedWriteAheadLog - clean old logs") { - logCleanUpTest(waitForCompletion = false) - } - - test("FileBasedWriteAheadLog - clean old logs synchronously") { - logCleanUpTest(waitForCompletion = true) - } - - private def logCleanUpTest(waitForCompletion: Boolean): Unit = { - // Write data with manager, recover with new manager and verify - val manualClock = new ManualClock - val dataToWrite = generateRandomData() - writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false) - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - - writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion) - - if (waitForCompletion) { - assert(getLogFilesInDirectory(testDir).size < logFiles.size) - } else { - eventually(timeout(1 second), interval(10 milliseconds)) { - assert(getLogFilesInDirectory(testDir).size < logFiles.size) - } - } - } - - test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") { - // Generate a set of log files - val manualClock = new ManualClock - val dataToWrite1 = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock) - val logFiles1 = getLogFilesInDirectory(testDir) - assert(logFiles1.size > 1) - - - // Recover old files and generate a second set of log files - val dataToWrite2 = generateRandomData() - manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock) - val logFiles2 = getLogFilesInDirectory(testDir) - assert(logFiles2.size > logFiles1.size) - - // Read the files and verify that all the written data can be read - val readData1 = readDataUsingWriteAheadLog(testDir) - assert(readData1 === (dataToWrite1 ++ dataToWrite2)) - - // Corrupt the first set of files so that they are basically unreadable - logFiles1.foreach { f => - val raf = new FileOutputStream(f, true).getChannel() - raf.truncate(1) - raf.close() - } +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests { - // Verify that the corrupted files do not prevent reading of the second set of data - val readData = readDataUsingWriteAheadLog(testDir) - assert(readData === dataToWrite2) - } + override val closeFileAfterWrite: Boolean = false + override val allowBatching: Boolean = true +} - test("FileBasedWriteAheadLog - do not create directories or files unless write") { - val nonexistentTempPath = File.createTempFile("test", "") - nonexistentTempPath.delete() - assert(!nonexistentTempPath.exists()) +class BatchedWithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { + override val closeFileAfterWrite: Boolean = true + override val allowBatching: Boolean = true +} - val writtenSegment = writeDataManually(generateRandomData(), testFile) - val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath, - new Configuration(), 1, 1, closeFileAfterWrite = false) - assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") - wal.read(writtenSegment.head) - assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") - } +class WithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { + override val closeFileAfterWrite: Boolean = true + override val allowBatching: Boolean = false } object WriteAheadLogSuite { @@ -373,18 +399,20 @@ object WriteAheadLogSuite { manualClock: ManualClock = new ManualClock, closeLog: Boolean = true, clockAdvanceTime: Int = 500, - closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = { + closeFileAfterWrite: Boolean = false, + allowBatching: Boolean = false): WriteAheadLog = { if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, closeFileAfterWrite) + val effectiveWal = if (allowBatching) new BatchedWriteAheadLog(wal) else wal // Ensure that 500 does not get sorted after 2000, so put a high base value. data.foreach { item => manualClock.advance(clockAdvanceTime) - wal.write(item, manualClock.getTimeMillis()) + effectiveWal.write(item, manualClock.getTimeMillis()) } - if (closeLog) wal.close() - wal + if (closeLog) effectiveWal.close() + effectiveWal } /** Read data from a segments of a log file directly and return the list of byte buffers. */ From e288cb2382d32ced61e7ae6f81a3b7cca2815eae Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 29 Oct 2015 12:02:52 -0700 Subject: [PATCH 10/26] more updates --- .../scheduler/ReceivedBlockTracker.scala | 4 +- .../streaming/util/BatchedWriteAheadLog.scala | 34 +++++++++- .../streaming/util/WriteAheadLogSuite.scala | 66 ++++++++++++++----- 3 files changed, 82 insertions(+), 22 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 4800d801fd2c8..56eba9806d62b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.streaming.Time -import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} +import org.apache.spark.streaming.util.{BatchedWriteAheadLog, WriteAheadLog, WriteAheadLogUtils} import org.apache.spark.util.{Clock, Utils} import org.apache.spark.{Logging, SparkConf} @@ -222,7 +222,7 @@ private[streaming] class ReceivedBlockTracker( def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { event match { case CombinedReceivedBlockTrackerLogEvent(events) => - events.foreach(r => resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent](r))) + BatchedWriteAheadLog.deaggregate(events).foreach(resolveEvent) case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 8c46b8f7ff022..a42a539ffcab9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.streaming.util import java.nio.ByteBuffer @@ -11,7 +28,7 @@ import scala.concurrent.duration._ import scala.util.control.NonFatal import org.apache.spark.Logging -import org.apache.spark.streaming.scheduler.CombinedReceivedBlockTrackerLogEvent +import org.apache.spark.streaming.scheduler.{ReceivedBlockTrackerLogEvent, CombinedReceivedBlockTrackerLogEvent} import org.apache.spark.util.{Utils, ThreadUtils} /** @@ -148,8 +165,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) // we take the latest record for the time to ensure that we don't clean up files earlier // than the expiration date of the records val time = buffer.last.time - segment = parent.write(ByteBuffer.wrap(Utils.serialize( - CombinedReceivedBlockTrackerLogEvent(buffer.map(_.record.array()).toArray))), time) + segment = parent.write(BatchedWriteAheadLog.aggregateRecords(buffer), time) } buffer.foreach(_.promise.success(segment)) } catch { @@ -161,3 +177,15 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) } } } + +private[streaming] object BatchedWriteAheadLog { + private[streaming] def aggregateRecords(records: Seq[RecordBuffer]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize( + CombinedReceivedBlockTrackerLogEvent(records.map(_.record.array()).toArray))) + } + + private[streaming] def deaggregate( + batchedEvents: Array[Array[Byte]]): Array[ReceivedBlockTrackerLogEvent] = { + batchedEvents.map(Utils.deserialize[ReceivedBlockTrackerLogEvent]) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index ff22dcbf5057e..493e434b9a969 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.Path import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter +import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} @@ -118,7 +119,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(fileSystem.exists(logDirectoryPath) === true) // Read data using manager and verify - val readData = readDataUsingWriteAheadLog(testDir) + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) assert(readData === writtenData) } @@ -128,7 +129,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte writeDataUsingWriteAheadLog(testDir, dataToWrite) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - val readData = readDataUsingWriteAheadLog(testDir) + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) assert(dataToWrite === readData) } @@ -176,7 +177,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(logFiles2.size > logFiles1.size) // Read the files and verify that all the written data can be read - val readData1 = readDataUsingWriteAheadLog(testDir) + val readData1 = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) assert(readData1 === (dataToWrite1 ++ dataToWrite2)) // Corrupt the first set of files so that they are basically unreadable @@ -187,7 +188,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte } // Verify that the corrupted files do not prevent reading of the second set of data - val readData = readDataUsingWriteAheadLog(testDir) + val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) assert(readData === dataToWrite2) } @@ -312,14 +313,35 @@ class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { val writtenData = logFiles.flatMap { file => readDataManually(file)} assert(writtenData === dataToWrite) } +} + +abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTests { + + test("serializing and deserializing batched records") { + val events = Seq( + BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)), + BatchAllocationEvent(null, null), + BatchCleanupEvent(Nil) + ) + val buffers = events.map(e => RecordBuffer(ByteBuffer.wrap(Utils.serialize(e)), 0L, null)) + val batched = BatchedWriteAheadLog.aggregateRecords(buffers) + val crbte = Utils.deserialize[CombinedReceivedBlockTrackerLogEvent](batched.array()) + val deaggregate = BatchedWriteAheadLog.deaggregate(crbte.events) + + assert(deaggregate.toSeq === events) + } +} + +trait CloseWriteAheadLogAfterWriteTests extends CommonWriteAheadLogTests { + import WriteAheadLogSuite._ test("FileBasedWriteAheadLog - close after write flag") { // Write data with rotation using WriteAheadLog class val numFiles = 3 val dataToWrite = Seq.tabulate(numFiles)(_.toString) // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, - closeFileAfterWrite = true) + closeFileAfterWrite = closeFileAfterWrite) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) @@ -329,18 +351,19 @@ class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { } } -class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests { +class BatchedWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { override val closeFileAfterWrite: Boolean = false override val allowBatching: Boolean = true } -class BatchedWithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { +class BatchedWithFileCloseWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests + with CloseWriteAheadLogAfterWriteTests { override val closeFileAfterWrite: Boolean = true override val allowBatching: Boolean = true } -class WithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { +class WithFileCloseWriteAheadLogSuite extends CloseWriteAheadLogAfterWriteTests { override val closeFileAfterWrite: Boolean = true override val allowBatching: Boolean = false } @@ -402,17 +425,15 @@ object WriteAheadLogSuite { closeFileAfterWrite: Boolean = false, allowBatching: Boolean = false): WriteAheadLog = { if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite) - val effectiveWal = if (allowBatching) new BatchedWriteAheadLog(wal) else wal + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) // Ensure that 500 does not get sorted after 2000, so put a high base value. data.foreach { item => manualClock.advance(clockAdvanceTime) - effectiveWal.write(item, manualClock.getTimeMillis()) + wal.write(item, manualClock.getTimeMillis()) } - if (closeLog) effectiveWal.close() - effectiveWal + if (closeLog) wal.close() + wal } /** Read data from a segments of a log file directly and return the list of byte buffers. */ @@ -462,9 +483,11 @@ object WriteAheadLogSuite { } /** Read all the data in the log file in a directory using the WriteAheadLog class. */ - def readDataUsingWriteAheadLog(logDirectory: String): Seq[String] = { - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite = false) + def readDataUsingWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): Seq[String] = { + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) val data = wal.readAll().asScala.map(byteBufferToString).toSeq wal.close() data @@ -486,6 +509,15 @@ object WriteAheadLogSuite { } } + def createWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): WriteAheadLog = { + val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, + closeFileAfterWrite) + if (allowBatching) new BatchedWriteAheadLog(wal) else wal + } + def generateRandomData(): Seq[String] = { (1 to 100).map { _.toString } } From 81e7f93ee67c31ea55f21808868ebf3081e54782 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 30 Oct 2015 12:57:07 -0700 Subject: [PATCH 11/26] ready for testing, not review --- .../scheduler/ReceivedBlockTracker.scala | 62 +++----- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../streaming/util/BatchedWriteAheadLog.scala | 145 +++++++++--------- .../streaming/util/WriteAheadLogUtils.scala | 6 +- 4 files changed, 103 insertions(+), 112 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 56eba9806d62b..738cba863021e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -41,9 +41,6 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: extends ReceivedBlockTrackerLogEvent private[streaming] case class BatchCleanupEvent(times: Seq[Time]) extends ReceivedBlockTrackerLogEvent -// Each event is in serialized format inside the array -private[streaming] case class CombinedReceivedBlockTrackerLogEvent( - events: Array[Array[Byte]]) extends ReceivedBlockTrackerLogEvent /** Class representing the blocks of all the streams allocated to a batch */ private[streaming] @@ -89,7 +86,9 @@ private[streaming] class ReceivedBlockTracker( try { val writeResult = writeToLog(BlockAdditionEvent(receivedBlockInfo)) if (writeResult) { - afterBlockAddAcknowledged(receivedBlockInfo) + synchronized { + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + } logDebug(s"Stream ${receivedBlockInfo.streamId} received " + s"block ${receivedBlockInfo.blockStoreResult.blockId}") } else { @@ -104,23 +103,19 @@ private[streaming] class ReceivedBlockTracker( } } - /** Update in-memory state after block add event has been logged to WAL. */ - private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = synchronized { - getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo - } - /** * Allocate all unallocated blocks to the given batch. * This event will get written to the write ahead log (if enabled). */ - def allocateBlocksToBatch(batchTime: Time): Unit = { + def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { val streamIdToBlocks = streamIds.map { streamId => (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { - afterBatchAllocationAcknowledged(batchTime, allocatedBlocks) + timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + lastAllocatedBatchTime = batchTime } else { logInfo(s"Possibly processed batch $batchTime need to be processed again in WAL recovery") } @@ -136,14 +131,6 @@ private[streaming] class ReceivedBlockTracker( } } - /** Update in-memory state after batch allocation event has been logged to WAL. */ - private def afterBatchAllocationAcknowledged( - batchTime: Time, - allocatedBlocks: AllocatedBlocks): Unit = synchronized { - timeToAllocatedBlocks.put(batchTime, allocatedBlocks) - lastAllocatedBatchTime = batchTime - } - /** Get the blocks allocated to the given batch. */ def getBlocksOfBatch(batchTime: Time): Map[Int, Seq[ReceivedBlockInfo]] = synchronized { timeToAllocatedBlocks.get(batchTime).map { _.streamIdToAllocatedBlocks }.getOrElse(Map.empty) @@ -201,7 +188,7 @@ private[streaming] class ReceivedBlockTracker( def insertAddedBlock(receivedBlockInfo: ReceivedBlockInfo) { logTrace(s"Recovery: Inserting added block $receivedBlockInfo") receivedBlockInfo.setBlockIdInvalid() - afterBlockAddAcknowledged(receivedBlockInfo) + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo } // Insert the recovered block-to-batch allocations and clear the queue of received blocks @@ -210,7 +197,8 @@ private[streaming] class ReceivedBlockTracker( logTrace(s"Recovery: Inserting allocated batch for time $batchTime to " + s"${allocatedBlocks.streamIdToAllocatedBlocks}") streamIdToUnallocatedBlockQueues.values.foreach { _.clear() } - afterBatchAllocationAcknowledged(batchTime, allocatedBlocks) + timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + lastAllocatedBatchTime = batchTime } // Cleanup the batch allocations @@ -219,35 +207,31 @@ private[streaming] class ReceivedBlockTracker( timeToAllocatedBlocks --= batchTimes } - def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = { - event match { - case CombinedReceivedBlockTrackerLogEvent(events) => - BatchedWriteAheadLog.deaggregate(events).foreach(resolveEvent) - case BlockAdditionEvent(receivedBlockInfo) => - insertAddedBlock(receivedBlockInfo) - case BatchAllocationEvent(time, allocatedBlocks) => - insertAllocatedBatch(time, allocatedBlocks) - case BatchCleanupEvent(batchTimes) => - cleanupBatches(batchTimes) - } - } - writeAheadLogOption.foreach { writeAheadLog => logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") writeAheadLog.readAll().asScala.foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) - resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent]( - byteBuffer.array, Thread.currentThread().getContextClassLoader)) + Utils.deserialize[ReceivedBlockTrackerLogEvent]( + byteBuffer.array, Thread.currentThread().getContextClassLoader) match { + case BlockAdditionEvent(receivedBlockInfo) => + insertAddedBlock(receivedBlockInfo) + case BatchAllocationEvent(time, allocatedBlocks) => + insertAllocatedBatch(time, allocatedBlocks) + case BatchCleanupEvent(batchTimes) => + cleanupBatches(batchTimes) + } } } } /** Write an update to the tracker to the write ahead log */ private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { - if (!isWriteAheadLogEnabled) return true - writeAheadLogOption.exists { logManager => + if (isWriteAheadLogEnabled) { logTrace(s"Writing record: $record") - logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) != null + writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), + clock.getTimeMillis()) != null + } else { + true } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 8e729183a9257..bb7adcc76833c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -491,7 +491,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false registerReceiver(streamId, typ, host, executorId, receiverEndpoint, context.senderAddress) context.reply(successful) case AddBlock(receivedBlockInfo) => - if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) { + if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool) f.onComplete(result => context.reply(result.get))(walBatchingThreadPool) } else { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index a42a539ffcab9..f1699e77c7ff7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -28,19 +28,8 @@ import scala.concurrent.duration._ import scala.util.control.NonFatal import org.apache.spark.Logging -import org.apache.spark.streaming.scheduler.{ReceivedBlockTrackerLogEvent, CombinedReceivedBlockTrackerLogEvent} import org.apache.spark.util.{Utils, ThreadUtils} -/** - * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled with - * the timestamp for the write request of the record, and the promise that will block the write - * request, while a separate thread is actually performing the write. - */ -private[util] case class RecordBuffer( - record: ByteBuffer, - time: Long, - promise: Promise[WriteAheadLogRecordHandle]) - /** * A wrapper for a WriteAheadLog that batches records before writing data. All other methods will * be passed on to the wrapped class. @@ -48,6 +37,16 @@ private[util] case class RecordBuffer( private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) extends WriteAheadLog with Logging { + /** + * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled with + * the timestamp for the write request of the record, and the promise that will block the write + * request, while a separate thread is actually performing the write. + */ + private[util] case class RecordBuffer( + record: ByteBuffer, + time: Long, + promise: Promise[WriteAheadLogRecordHandle]) + /** A thread pool for fulfilling log write promises */ private val batchWriterThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("wal-batch-writer-thead-pool")) @@ -57,7 +56,11 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds - private val writeAheadLogBatchWriter: BatchedLogWriter = startBatchedWriterThread() + // Whether the writer thread is active + private var active: Boolean = true + private val buffer = new ArrayBuffer[RecordBuffer]() + + startBatchedWriterThread() /** * Write a byte buffer to the log file. This method adds the byteBuffer to a queue and blocks @@ -78,7 +81,8 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) } /** - * Read a segment from an existing Write Ahead Log. + * Read a segment from an existing Write Ahead Log. The data may be aggregated, and the user + * should de-aggregate using [[BatchedWriteAheadLog.deaggregate]] * * This method is handled by the parent WriteAheadLog. */ @@ -92,7 +96,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) * This method is handled by the parent WriteAheadLog. */ override def readAll(): JIterator[ByteBuffer] = { - parent.readAll() + parent.readAll().asScala.flatMap(BatchedWriteAheadLog.deaggregate).asJava } /** @@ -106,86 +110,89 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) /** - * Stop the manager, close any open log writer. - * - * This method is handled by the parent WriteAheadLog. + * Stop the batched writer thread, fulfill promises with failures and close parent writer. */ override def close(): Unit = { - writeAheadLogBatchWriter.stop() + logInfo("BatchedWriteAheadLog shutting down.") + active = false + fulfillPromises() batchWriterThreadPool.shutdownNow() parent.close() } - /** Start the actual log writer on a separate thread. */ - private def startBatchedWriterThread(): BatchedLogWriter = { - val writer = new BatchedLogWriter() - val thread = new Thread(writer, "Batched WAL Writer") - thread.setDaemon(true) - thread.start() - writer + /** + * Respond to any promises that may have been left in the queue, to unblock receivers during + * shutdown. + */ + private def fulfillPromises(): Unit = { + while (!walWriteQueue.isEmpty) { + val RecordBuffer(_, _, promise) = walWriteQueue.poll() + promise.success(null) + } } - /** A helper class that writes LogEvents in a separate thread to allow for batching. */ - private[util] class BatchedLogWriter extends Runnable { - - private var active: Boolean = true - private val buffer = new ArrayBuffer[RecordBuffer]() - - override def run(): Unit = { + /** Start the actual log writer on a separate thread. Visible(protected) for testing. */ + protected def startBatchedWriterThread(): Unit = { + ThreadUtils.runInNewThread("Batched WAL Writer", isDaemon = true) { while (active) { try { flushRecords() } catch { case NonFatal(e) => - logError("Exception while flushing records in Batch Write Ahead Log writer.", e) + logWarning("Encountered exception in Batched Writer Thread.", e) } } - logInfo("Batch Write Ahead Log writer shutting down.") + logInfo("Batched WAL Writer thread exiting.") } + } - def stop(): Unit = { - logInfo("Stopping Batch Write Ahead Log writer.") - active = false + /** Write all the records in the buffer to the write ahead log. Visible for testing. */ + protected def flushRecords(): Unit = { + try { + buffer.append(walWriteQueue.take()) + val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 + logDebug(s"Received $numBatched records from queue") + } catch { + case _: InterruptedException => + logWarning("Batch Write Ahead Log Writer queue interrupted.") } - - /** Write all the records in the buffer to the write ahead log. */ - private def flushRecords(): Unit = { - try { - buffer.append(walWriteQueue.take()) - val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 - logDebug(s"Received $numBatched records from queue") - } catch { - case _: InterruptedException => - logWarning("Batch Write Ahead Log Writer queue interrupted.") - } - try { - var segment: WriteAheadLogRecordHandle = null - if (buffer.length > 0) { - logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") - // we take the latest record for the time to ensure that we don't clean up files earlier - // than the expiration date of the records - val time = buffer.last.time - segment = parent.write(BatchedWriteAheadLog.aggregateRecords(buffer), time) - } - buffer.foreach(_.promise.success(segment)) - } catch { - case NonFatal(e) => - logWarning(s"Batch WAL Writer failed to write $buffer", e) - buffer.foreach(_.promise.success(null)) + try { + var segment: WriteAheadLogRecordHandle = null + if (buffer.length > 0) { + logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") + // we take the latest record for the time to ensure that we don't clean up files earlier + // than the expiration date of the records + val time = buffer.last.time + segment = parent.write(BatchedWriteAheadLog.aggregate(buffer), time) } - buffer.clear() + buffer.foreach(_.promise.success(segment)) + } catch { + case NonFatal(e) => + logWarning(s"Batch WAL Writer failed to write $buffer", e) + buffer.foreach(_.promise.success(null)) } + buffer.clear() } } +/** Static methods for aggregating and de-aggregating records. */ private[streaming] object BatchedWriteAheadLog { - private[streaming] def aggregateRecords(records: Seq[RecordBuffer]): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize( - CombinedReceivedBlockTrackerLogEvent(records.map(_.record.array()).toArray))) + /** Aggregate multiple serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. */ + private[streaming] def aggregate(records: Seq[RecordBuffer]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(records.map(_.record.array().toArray))) } - private[streaming] def deaggregate( - batchedEvents: Array[Array[Byte]]): Array[ReceivedBlockTrackerLogEvent] = { - batchedEvents.map(Utils.deserialize[ReceivedBlockTrackerLogEvent]) + /** + * De-aggregate serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. + * A stream may not have used batching initially, but started using it after a restart. This + * method therefore needs to be backwards compatible. + */ + private[streaming] def deaggregate(buffer: ByteBuffer): Array[ByteBuffer] = { + try { + Utils.deserialize[Array[Array[Byte]]](buffer.array()).map(ByteBuffer.wrap) + } catch { + case _: ClassCastException => // users may restart a stream with batching enabled + Array(buffer) + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index ba93019fa9dca..34d1ee66cb9e7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -65,8 +65,8 @@ private[streaming] object WriteAheadLogUtils extends Logging { } } - def isBatchingEnabled(conf: SparkConf): Boolean = { - conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) + def isBatchingEnabled(conf: SparkConf, isDriver: Boolean): Boolean = { + isDriver && conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) } def shouldCloseFileAfterWrite(conf: SparkConf, isDriver: Boolean): Boolean = { @@ -133,7 +133,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { getRollingIntervalSecs(sparkConf, isDriver), getMaxFailures(sparkConf, isDriver), shouldCloseFileAfterWrite(sparkConf, isDriver)) } - if (isDriver && isBatchingEnabled(sparkConf)) { + if (isBatchingEnabled(sparkConf, isDriver)) { new BatchedWriteAheadLog(wal) } else { wal From 4d107954541bd662671a3b6d9ad89bba83a5a984 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 3 Nov 2015 08:55:53 -0800 Subject: [PATCH 12/26] updated tests --- .../scheduler/ReceivedBlockTracker.scala | 2 +- .../streaming/util/BatchedWriteAheadLog.scala | 52 +-- .../streaming/ReceivedBlockTrackerSuite.scala | 93 ------ .../streaming/util/WriteAheadLogSuite.scala | 314 +++++++++++++----- 4 files changed, 266 insertions(+), 195 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 738cba863021e..0b2339b25b18f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -228,7 +228,7 @@ private[streaming] class ReceivedBlockTracker( private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { logTrace(s"Writing record: $record") - writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), + writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis()) != null } else { true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index f1699e77c7ff7..ff279bd50ce48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -37,15 +37,7 @@ import org.apache.spark.util.{Utils, ThreadUtils} private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) extends WriteAheadLog with Logging { - /** - * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled with - * the timestamp for the write request of the record, and the promise that will block the write - * request, while a separate thread is actually performing the write. - */ - private[util] case class RecordBuffer( - record: ByteBuffer, - time: Long, - promise: Promise[WriteAheadLogRecordHandle]) + import BatchedWriteAheadLog._ /** A thread pool for fulfilling log write promises */ private val batchWriterThreadPool = ExecutionContext.fromExecutorService( @@ -58,7 +50,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) // Whether the writer thread is active private var active: Boolean = true - private val buffer = new ArrayBuffer[RecordBuffer]() + protected val buffer = new ArrayBuffer[RecordBuffer]() startBatchedWriterThread() @@ -96,7 +88,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) * This method is handled by the parent WriteAheadLog. */ override def readAll(): JIterator[ByteBuffer] = { - parent.readAll().asScala.flatMap(BatchedWriteAheadLog.deaggregate).asJava + parent.readAll().asScala.flatMap(deaggregate).asJava } /** @@ -131,19 +123,23 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) } } - /** Start the actual log writer on a separate thread. Visible(protected) for testing. */ + /** Start the actual log writer on a separate thread. Visible (protected) for testing. */ protected def startBatchedWriterThread(): Unit = { - ThreadUtils.runInNewThread("Batched WAL Writer", isDaemon = true) { - while (active) { - try { - flushRecords() - } catch { - case NonFatal(e) => - logWarning("Encountered exception in Batched Writer Thread.", e) + val thread = new Thread(new Runnable { + override def run(): Unit = { + while (active) { + try { + flushRecords() + } catch { + case NonFatal(e) => + logWarning("Encountered exception in Batched Writer Thread.", e) + } } + logInfo("Batched WAL Writer thread exiting.") } - logInfo("Batched WAL Writer thread exiting.") - } + }, "Batched WAL Writer") + thread.setDaemon(true) + thread.start() } /** Write all the records in the buffer to the write ahead log. Visible for testing. */ @@ -163,7 +159,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) // we take the latest record for the time to ensure that we don't clean up files earlier // than the expiration date of the records val time = buffer.last.time - segment = parent.write(BatchedWriteAheadLog.aggregate(buffer), time) + segment = parent.write(aggregate(buffer), time) } buffer.foreach(_.promise.success(segment)) } catch { @@ -177,9 +173,19 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) /** Static methods for aggregating and de-aggregating records. */ private[streaming] object BatchedWriteAheadLog { + /** + * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled + * with the timestamp for the write request of the record, and the promise that will block the + * write request, while a separate thread is actually performing the write. + */ + private[util] case class RecordBuffer( + record: ByteBuffer, + time: Long, + promise: Promise[WriteAheadLogRecordHandle]) + /** Aggregate multiple serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. */ private[streaming] def aggregate(records: Seq[RecordBuffer]): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize(records.map(_.record.array().toArray))) + ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(_.record.array()).toArray)) } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 69d103443b4bb..0d47d85ba0625 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -267,96 +267,3 @@ class ReceivedBlockTrackerSuite implicit def timeToMillis(time: Time): Long = time.milliseconds } - -/** -class AsyncReceivedBlockTrackerSuite extends ReceivedBlockTrackerSuite with MockitoSugar { - override val isBatchingEnabled = true - - - private class TestRBT( - clock: Clock = new SystemClock, - cpDirOption: Option[String] = Some(checkpointDirectory.toString), - recoverFromWriteAheadLog: Boolean = true) - extends ReceivedBlockTracker(conf, hadoopConf, Seq(streamId), clock, - recoverFromWriteAheadLog, cpDirOption) { - - override def createBatchWriteAheadLogWriter(): Option[BatchLogWriter] = None - - def updateWALWriteStatus(event: ReceivedBlockTrackerLogEvent, successful: Boolean): Unit = { - val promise = walWriteStatusMap.get(event) - promise.success(successful) - walWriteQueue.poll() - } - - def getQueueLength(): Int = walWriteQueue.size() - - def addToQueue(event: ReceivedBlockTrackerLogEvent): Boolean = { - writeToLog(event) - } - } - Class that will help us test batching. - - private def waitUntilTrue(f: () => Int, value: Int): Boolean = { - val timeOut = 2000 - val start = System.currentTimeMillis() - var result = false - while (!result && (System.currentTimeMillis() - start) < timeOut) { - Thread.sleep(50) - result = f() == value - } - result - } - - test("records get added to a queue") { - val numSuccess = new AtomicInteger() - val numFail = new AtomicInteger() - val rbt = new TestRBT() - - def getNumSuccess(): Int = numSuccess.get() - def getNumFail(): Int = numFail.get() - - val walBatchingThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) - - def eventFuture(event: ReceivedBlockTrackerLogEvent): Unit = { - val f = Future(rbt.addToQueue(event))(walBatchingThreadPool) - f.onSuccess { case v => - if (v) numSuccess.incrementAndGet() else numFail.incrementAndGet() - }(walBatchingThreadPool) - } - - assert(rbt.getQueueLength === 0) - val event1 = BlockAdditionEvent(ReceivedBlockInfo(1, None, None, null)) - val event2 = BlockAdditionEvent(null) - val event3 = BatchAllocationEvent(null, null) - val event4 = BlockAdditionEvent(ReceivedBlockInfo(2, None, None, null)) - val event5 = BatchCleanupEvent(Nil) - - eventFuture(event1) - assert(waitUntilTrue(rbt.getQueueLength, 1)) - assert(numSuccess.get() === 0) - assert(numFail.get() === 0) - - rbt.updateWALWriteStatus(event1, successful = false) - assert(waitUntilTrue(getNumFail, 1)) - assert(waitUntilTrue(rbt.getQueueLength, 0)) - - eventFuture(event2) - eventFuture(event3) - eventFuture(event4) - assert(waitUntilTrue(rbt.getQueueLength, 3)) - rbt.updateWALWriteStatus(event2, successful = true) - rbt.updateWALWriteStatus(event3, successful = true) - assert(waitUntilTrue(getNumSuccess, 2)) - assert(waitUntilTrue(rbt.getQueueLength, 1)) - - eventFuture(event5) - assert(waitUntilTrue(rbt.getQueueLength, 2)) - rbt.updateWALWriteStatus(event4, successful = true) - rbt.updateWALWriteStatus(event5, successful = true) - assert(waitUntilTrue(getNumSuccess, 4)) - assert(waitUntilTrue(rbt.getQueueLength, 0)) - } - -} - */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 493e434b9a969..376bdd4da4dff 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -19,20 +19,29 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer import java.util +import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{Await, Promise, Future, ExecutionContext} import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.mockito.Matchers._ +import org.mockito.Matchers.{eq => meq} +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar.{mock => mmock} import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.util.{ThreadUtils, ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfter { @@ -61,51 +70,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte Utils.deleteRecursively(tempDir) } - test("WriteAheadLogUtils - log selection and creation") { - val logDir = Utils.createTempDir().getAbsolutePath() - - def assertDriverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log - } - - def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log - } - - val emptyConf = new SparkConf() // no log configuration - assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) - assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) - - // Verify setting driver WAL class - val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[MockWriteAheadLog0](conf1) - assertReceiverLogClass[FileBasedWriteAheadLog](conf1) - - // Verify setting receiver WAL class - val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) - assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) - - // Verify setting receiver WAL class with 1-arg constructor - val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog1].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) - - // Verify failure setting receiver WAL class with 2-arg constructor - intercept[SparkException] { - val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog2].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) - } - } - - test("WriteAheadLog - read rotating logs") { + test("read rotating logs") { // Write data manually for testing reading through WriteAheadLog val writtenData = (1 to 10).map { i => val data = generateRandomData() @@ -123,7 +88,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(readData === writtenData) } - test("WriteAheadLog - recover past logs when creating new manager") { + test("recover past logs when creating new manager") { // Write data with manager, recover with new manager and verify val dataToWrite = generateRandomData() writeDataUsingWriteAheadLog(testDir, dataToWrite) @@ -133,11 +98,11 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(dataToWrite === readData) } - test("WriteAheadLog - clean old logs") { + test("clean old logs") { logCleanUpTest(waitForCompletion = false) } - test("WriteAheadLog - clean old logs synchronously") { + test("clean old logs synchronously") { logCleanUpTest(waitForCompletion = true) } @@ -145,7 +110,8 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() - writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false) + writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false, + closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) @@ -154,17 +120,18 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) } else { - eventually(timeout(1 second), interval(10 milliseconds)) { + eventually(Eventually.timeout(1 second), interval(10 milliseconds)) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) } } } - test("WriteAheadLog - handling file errors while reading rotating logs") { + test("handling file errors while reading rotating logs") { // Generate a set of log files val manualClock = new ManualClock val dataToWrite1 = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock) + writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock, + closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) val logFiles1 = getLogFilesInDirectory(testDir) assert(logFiles1.size > 1) @@ -172,7 +139,8 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock) + writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock, + closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) @@ -192,7 +160,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(readData === dataToWrite2) } - test("WriteAheadLog - do not create directories or files unless write") { + test("do not create directories or files unless write") { val nonexistentTempPath = File.createTempFile("test", "") nonexistentTempPath.delete() assert(!nonexistentTempPath.exists()) @@ -213,6 +181,50 @@ class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { override protected val closeFileAfterWrite: Boolean = false override protected val allowBatching: Boolean = false + test("WriteAheadLogUtils - log selection and creation") { + val logDir = Utils.createTempDir().getAbsolutePath() + + def assertDriverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + val emptyConf = new SparkConf() // no log configuration + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](conf1) + assertReceiverLogClass[FileBasedWriteAheadLog](conf1) + + // Verify setting receiver WAL class + val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + + // Verify setting receiver WAL class with 1-arg constructor + val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog1].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + + // Verify failure setting receiver WAL class with 2-arg constructor + intercept[SparkException] { + val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog2].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + } + } + test("FileBasedWriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) @@ -305,18 +317,21 @@ class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { test("FileBasedWriteAheadLog - write rotating logs") { // Write data with rotation using WriteAheadLog class val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite) + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite = closeFileAfterWrite, + allowBatching = allowBatching) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - val writtenData = logFiles.flatMap { file => readDataManually(file)} + val writtenData = logFiles.flatMap { file => readDataManually[String](file)} assert(writtenData === dataToWrite) } } abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTests { + import BatchedWriteAheadLog._ + test("serializing and deserializing batched records") { val events = Seq( BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)), @@ -325,15 +340,149 @@ abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTest ) val buffers = events.map(e => RecordBuffer(ByteBuffer.wrap(Utils.serialize(e)), 0L, null)) - val batched = BatchedWriteAheadLog.aggregateRecords(buffers) - val crbte = Utils.deserialize[CombinedReceivedBlockTrackerLogEvent](batched.array()) - val deaggregate = BatchedWriteAheadLog.deaggregate(crbte.events) + val batched = BatchedWriteAheadLog.aggregate(buffers) + val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer => + Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array())) assert(deaggregate.toSeq === events) } + + // Class that will help us test batching. + private class MockBatchedWriteAheadLog(parent: WriteAheadLog) + extends BatchedWriteAheadLog(parent) { + + override def startBatchedWriterThread(): Unit = {} + + override def flushRecords(): Unit = { + buffer.append(walWriteQueue.take()) + walWriteQueue.drainTo(buffer.asJava) + } + + def mockWrite(successful: Boolean): Seq[RecordBuffer] = { + val records = buffer.toSeq + buffer.foreach { case RecordBuffer(byteBuffer, time, promise) => + if (successful) promise.success(mmock[WriteAheadLogRecordHandle]) else promise.success(null) + } + buffer.clear() + records + } + + def getQueueLength(): Int = walWriteQueue.size() + } + + + private def waitUntilTrue(f: () => Int, value: Int): Boolean = { + val timeOut = 2000 + val start = System.currentTimeMillis() + var result = false + while (!result && (System.currentTimeMillis() - start) < timeOut) { + Thread.sleep(50) + result = f() == value + } + result + } + + import WriteAheadLogSuite._ + + test("records get added to a queue") { + val numSuccess = new AtomicInteger() + val numFail = new AtomicInteger() + val wal = + new MockBatchedWriteAheadLog(createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching)) + + def getNumSuccess(): Int = numSuccess.get() + def getNumFail(): Int = numFail.get() + + val walBatchingThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) + + def eventFuture(event: String, time: Long): Unit = { + val f = Future(wal.write(event, time))(walBatchingThreadPool) + f.onSuccess { case v => + if (v != null) numSuccess.incrementAndGet() else numFail.incrementAndGet() + }(walBatchingThreadPool) + } + + assert(wal.getQueueLength === 0) + val event1 = "hello" + val event2 = "world" + val event3 = "this" + val event4 = "is" + val event5 = "doge" + + eventFuture(event1, 5L) + assert(waitUntilTrue(wal.getQueueLength, 1)) + assert(numSuccess.get() === 0) + assert(numFail.get() === 0) + + wal.flushRecords() + wal.mockWrite(successful = false) + assert(waitUntilTrue(getNumFail, 1)) + assert(waitUntilTrue(wal.getQueueLength, 0)) + + eventFuture(event2, 10L) + eventFuture(event3, 11L) + eventFuture(event4, 12L) + assert(waitUntilTrue(wal.getQueueLength, 3)) + wal.flushRecords() + wal.mockWrite(successful = true) + assert(waitUntilTrue(wal.getQueueLength, 0)) + assert(waitUntilTrue(getNumSuccess, 3)) + + eventFuture(event5, 20L) + assert(waitUntilTrue(wal.getQueueLength, 1)) + wal.flushRecords() + wal.mockWrite(successful = true) + + assert(waitUntilTrue(wal.getQueueLength, 0)) + assert(waitUntilTrue(getNumSuccess, 4)) + } + + test("parent WriteAheadLog writes aggregated entries with the timestamp of last entry") { + val parentWAL = mmock[FileBasedWriteAheadLog] + val wal = new BatchedWriteAheadLog(parentWAL) + + val promise = Promise[Boolean]() + when(parentWAL.write(any[ByteBuffer], + any[Long])).thenAnswer(new Answer[FileBasedWriteAheadLogSegment] { + override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = { + Await.ready(promise.future, 2.seconds) + mmock[FileBasedWriteAheadLogSegment] + } + }) + + val event1 = "hello" + val event2 = "world" + val event3 = "this" + val event4 = "is" + val event5 = "doge" + + wal.write(event1, 3L) // 3 will automatically be flushed for the first write + wal.write(event2, 5L) + wal.write(event3, 8L) + wal.write(event4, 12L) + wal.write(event5, 10L) // rest of the records will be batched while it takes 3 to get written + promise.success(true) + + verify(parentWAL, times(1)).write(any[ByteBuffer], meq(3L)) + + Thread.sleep(100) + // 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. + verify(parentWAL, times(1)).write(any[ByteBuffer], meq(10L)) + } } -trait CloseWriteAheadLogAfterWriteTests extends CommonWriteAheadLogTests { +class BatchedWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { + + override val closeFileAfterWrite: Boolean = false + override val allowBatching: Boolean = true +} + +class BatchedWithFileCloseWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { + override val closeFileAfterWrite: Boolean = true + override val allowBatching: Boolean = true + import WriteAheadLogSuite._ test("FileBasedWriteAheadLog - close after write flag") { // Write data with rotation using WriteAheadLog class @@ -341,31 +490,40 @@ trait CloseWriteAheadLogAfterWriteTests extends CommonWriteAheadLogTests { val dataToWrite = Seq.tabulate(numFiles)(_.toString) // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, - closeFileAfterWrite = closeFileAfterWrite) + closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size === numFiles) - val writtenData = logFiles.flatMap { file => readDataManually(file)} + val writtenData: Seq[String] = logFiles.flatMap { file => + val data = readDataManually[Array[Array[Byte]]](file) + data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) + } assert(writtenData === dataToWrite) } } -class BatchedWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { +class WithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { + override val closeFileAfterWrite: Boolean = true + override val allowBatching: Boolean = false - override val closeFileAfterWrite: Boolean = false - override val allowBatching: Boolean = true -} + import WriteAheadLogSuite._ -class BatchedWithFileCloseWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests - with CloseWriteAheadLogAfterWriteTests { - override val closeFileAfterWrite: Boolean = true - override val allowBatching: Boolean = true -} + test("FileBasedWriteAheadLog - close after write flag") { + // Write data with rotation using WriteAheadLog class + val numFiles = 3 + val dataToWrite = Seq.tabulate(numFiles)(_.toString) + // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, + closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) -class WithFileCloseWriteAheadLogSuite extends CloseWriteAheadLogAfterWriteTests { - override val closeFileAfterWrite: Boolean = true - override val allowBatching: Boolean = false + // Read data manually to verify the written data + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size === numFiles) + val writtenData = logFiles.flatMap(file => readDataManually[String](file)) + + assert(writtenData === dataToWrite) + } } object WriteAheadLogSuite { @@ -455,16 +613,16 @@ object WriteAheadLogSuite { } /** Read all the data from a log file directly and return the list of byte buffers. */ - def readDataManually(file: String): Seq[String] = { + def readDataManually[T](file: String): Seq[T] = { val reader = HdfsUtils.getInputStream(file, hadoopConf) - val buffer = new ArrayBuffer[String] + val buffer = new ArrayBuffer[T] try { while (true) { // Read till EOF is thrown val length = reader.readInt() val bytes = new Array[Byte](length) reader.read(bytes) - buffer += Utils.deserialize[String](bytes) + buffer += Utils.deserialize[T](bytes) } } catch { case ex: EOFException => From c0063d3b5c611e85091ec86c8b6ee355c982cb21 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Wed, 4 Nov 2015 16:08:04 -0800 Subject: [PATCH 13/26] address 4 --- .../scheduler/ReceivedBlockTracker.scala | 6 +- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../streaming/util/BatchedWriteAheadLog.scala | 36 ++- .../streaming/util/WriteAheadLogSuite.scala | 278 ++++++++++-------- 4 files changed, 185 insertions(+), 137 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 0b2339b25b18f..fbe8ed6ce86bb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -228,8 +228,10 @@ private[streaming] class ReceivedBlockTracker( private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { logTrace(s"Writing record: $record") - writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), - clock.getTimeMillis()) != null + val handle = writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), + clock.getTimeMillis()) + + handle != null } else { true } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index bb7adcc76833c..b95d23654e589 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -607,7 +607,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def onStop(): Unit = { submitJobThreadPool.shutdownNow() - walBatchingThreadPool.shutdownNow() + walBatchingThreadPool.shutdown() } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index ff279bd50ce48..f1c74dba89754 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -33,26 +33,24 @@ import org.apache.spark.util.{Utils, ThreadUtils} /** * A wrapper for a WriteAheadLog that batches records before writing data. All other methods will * be passed on to the wrapped class. + * + * Parent exposed for testing. */ -private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog) extends WriteAheadLog with Logging { import BatchedWriteAheadLog._ - /** A thread pool for fulfilling log write promises */ - private val batchWriterThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("wal-batch-writer-thead-pool")) - // exposed for tests protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]() private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds // Whether the writer thread is active - private var active: Boolean = true + @volatile private var active: Boolean = true protected val buffer = new ArrayBuffer[RecordBuffer]() - startBatchedWriterThread() + private val batchedWriterThread = startBatchedWriterThread() /** * Write a byte buffer to the log file. This method adds the byteBuffer to a queue and blocks @@ -62,7 +60,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) val promise = Promise[WriteAheadLogRecordHandle]() walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise)) try { - Await.result(promise.future.recover { case _ => null }(batchWriterThreadPool), + Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread), WAL_WRITE_STATUS_TIMEOUT.milliseconds) } catch { case e: TimeoutException => @@ -107,8 +105,9 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) override def close(): Unit = { logInfo("BatchedWriteAheadLog shutting down.") active = false + batchedWriterThread.interrupt() + batchedWriterThread.join() fulfillPromises() - batchWriterThreadPool.shutdownNow() parent.close() } @@ -124,7 +123,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) } /** Start the actual log writer on a separate thread. Visible (protected) for testing. */ - protected def startBatchedWriterThread(): Unit = { + protected def startBatchedWriterThread(): Thread = { val thread = new Thread(new Runnable { override def run(): Unit = { while (active) { @@ -140,6 +139,7 @@ private[streaming] class BatchedWriteAheadLog(parent: WriteAheadLog) }, "Batched WAL Writer") thread.setDaemon(true) thread.start() + thread } /** Write all the records in the buffer to the write ahead log. Visible for testing. */ @@ -183,9 +183,17 @@ private[streaming] object BatchedWriteAheadLog { time: Long, promise: Promise[WriteAheadLogRecordHandle]) + /** Copies the byte array of a ByteBuffer. */ + private def getByteArray(buffer: ByteBuffer): Array[Byte] = { + val byteArray = new Array[Byte](buffer.remaining()) + buffer.get(byteArray) + byteArray + } + /** Aggregate multiple serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. */ - private[streaming] def aggregate(records: Seq[RecordBuffer]): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(_.record.array()).toArray)) + def aggregate(records: Seq[RecordBuffer]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]]( + records.map(recordBuffer => getByteArray(recordBuffer.record)).toArray)) } /** @@ -193,9 +201,9 @@ private[streaming] object BatchedWriteAheadLog { * A stream may not have used batching initially, but started using it after a restart. This * method therefore needs to be backwards compatible. */ - private[streaming] def deaggregate(buffer: ByteBuffer): Array[ByteBuffer] = { + def deaggregate(buffer: ByteBuffer): Array[ByteBuffer] = { try { - Utils.deserialize[Array[Array[Byte]]](buffer.array()).map(ByteBuffer.wrap) + Utils.deserialize[Array[Array[Byte]]](getByteArray(buffer)).map(ByteBuffer.wrap) } catch { case _: ClassCastException => // users may restart a stream with batching enabled Array(buffer) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 376bdd4da4dff..5ac9e0604b648 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -44,7 +44,90 @@ import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ThreadUtils, ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfter { +class WriteAheadLogUtilsSuite extends SparkFunSuite { + import WriteAheadLogSuite._ + + val logDir = Utils.createTempDir().getAbsolutePath() + + def assertDriverLogClass[T <: WriteAheadLog: ClassTag]( + conf: SparkConf, + isBatched: Boolean = false): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) + if (isBatched) { + assert(log.isInstanceOf[BatchedWriteAheadLog]) + val parentLog = log.asInstanceOf[BatchedWriteAheadLog].parent + assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass) + } else { + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + } + log + } + + def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + test("WriteAheadLogUtils - log selection and creation") { + + val emptyConf = new SparkConf() // no log configuration + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](conf1) + assertReceiverLogClass[FileBasedWriteAheadLog](conf1) + + // Verify setting receiver WAL class + val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + + // Verify setting receiver WAL class with 1-arg constructor + val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog1].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + + // Verify failure setting receiver WAL class with 2-arg constructor + intercept[SparkException] { + val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog2].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + } + } + + test("WriteAheadLogUtils - wrap WriteAheadLog in BatchedWriteAheadLog when batching is enabled") { + def getBatchedSparkConf: SparkConf = + new SparkConf().set("spark.streaming.driver.writeAheadLog.allowBatching", "true") + + val emptyConf = getBatchedSparkConf + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val conf1 = getBatchedSparkConf.set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](conf1, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](conf1) + + // Verify receivers are not wrapped + val receiverWALConf = getBatchedSparkConf.set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf, isBatched = true) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + } +} + +/** Common tests for WriteAheadLogs that we would like to test with different configurations. */ +abstract class CommonWriteAheadLogTests( + allowBatching: Boolean, + closeFileAfterWrite: Boolean, + testTag: String = "") + extends SparkFunSuite with BeforeAndAfter { import WriteAheadLogSuite._ @@ -53,8 +136,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte var testDir: String = null var testFile: String = null var writeAheadLog: WriteAheadLog = null - protected val allowBatching: Boolean - protected val closeFileAfterWrite: Boolean + protected def testPrefix = if (testTag != "") testTag + " - " else testTag before { tempDir = Utils.createTempDir() @@ -70,7 +152,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte Utils.deleteRecursively(tempDir) } - test("read rotating logs") { + test(testPrefix + "read rotating logs") { // Write data manually for testing reading through WriteAheadLog val writtenData = (1 to 10).map { i => val data = generateRandomData() @@ -88,7 +170,20 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(readData === writtenData) } - test("recover past logs when creating new manager") { + test(testPrefix + "write rotating logs") { + // Write data with rotation using WriteAheadLog class + val dataToWrite = generateRandomData() + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite = closeFileAfterWrite, + allowBatching = allowBatching) + + // Read data manually to verify the written data + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val writtenData = readAndDeserializeDataManually(logFiles, allowBatching) + assert(writtenData === dataToWrite) + } + + test(testPrefix + "recover past logs when creating new manager") { // Write data with manager, recover with new manager and verify val dataToWrite = generateRandomData() writeDataUsingWriteAheadLog(testDir, dataToWrite) @@ -98,11 +193,11 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(dataToWrite === readData) } - test("clean old logs") { + test(testPrefix + "clean old logs") { logCleanUpTest(waitForCompletion = false) } - test("clean old logs synchronously") { + test(testPrefix + "clean old logs synchronously") { logCleanUpTest(waitForCompletion = true) } @@ -126,7 +221,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte } } - test("handling file errors while reading rotating logs") { + test(testPrefix + "handling file errors while reading rotating logs") { // Generate a set of log files val manualClock = new ManualClock val dataToWrite1 = generateRandomData() @@ -160,7 +255,7 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte assert(readData === dataToWrite2) } - test("do not create directories or files unless write") { + test(testPrefix + "do not create directories or files unless write") { val nonexistentTempPath = File.createTempFile("test", "") nonexistentTempPath.delete() assert(!nonexistentTempPath.exists()) @@ -174,57 +269,11 @@ abstract class CommonWriteAheadLogTests extends SparkFunSuite with BeforeAndAfte } } -class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { +class FileBasedWriteAheadLogSuite + extends CommonWriteAheadLogTests(false, false, "FileBasedWriteAheadLog") { import WriteAheadLogSuite._ - override protected val closeFileAfterWrite: Boolean = false - override protected val allowBatching: Boolean = false - - test("WriteAheadLogUtils - log selection and creation") { - val logDir = Utils.createTempDir().getAbsolutePath() - - def assertDriverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log - } - - def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log - } - - val emptyConf = new SparkConf() // no log configuration - assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) - assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) - - // Verify setting driver WAL class - val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[MockWriteAheadLog0](conf1) - assertReceiverLogClass[FileBasedWriteAheadLog](conf1) - - // Verify setting receiver WAL class - val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) - assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) - - // Verify setting receiver WAL class with 1-arg constructor - val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog1].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) - - // Verify failure setting receiver WAL class with 2-arg constructor - intercept[SparkException] { - val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog2].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) - } - } - test("FileBasedWriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) @@ -313,23 +362,33 @@ class DefaultWriteAheadLogSuite extends CommonWriteAheadLogTests { } reader.close() } +} + +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String) + extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) { - test("FileBasedWriteAheadLog - write rotating logs") { + import WriteAheadLogSuite._ + test(testPrefix + "close after write flag") { // Write data with rotation using WriteAheadLog class - val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite = closeFileAfterWrite, - allowBatching = allowBatching) + val numFiles = 3 + val dataToWrite = Seq.tabulate(numFiles)(_.toString) + // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, + closeFileAfterWrite = true, allowBatching = allowBatching) // Read data manually to verify the written data val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size > 1) - val writtenData = logFiles.flatMap { file => readDataManually[String](file)} + assert(logFiles.size === numFiles) + val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching) assert(writtenData === dataToWrite) } } -abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTests { +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite + extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog") +/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */ +class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite { import BatchedWriteAheadLog._ test("serializing and deserializing batched records") { @@ -346,12 +405,21 @@ abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTest assert(deaggregate.toSeq === events) } +} + +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( + allowBatching = true, + closeFileAfterWrite = false, + "BatchedWriteAheadLog") { + + import BatchedWriteAheadLog._ // Class that will help us test batching. - private class MockBatchedWriteAheadLog(parent: WriteAheadLog) - extends BatchedWriteAheadLog(parent) { + private class MockBatchedWriteAheadLog( + parent: WriteAheadLog, + writerThread: Thread = mmock[Thread]) extends BatchedWriteAheadLog(parent) { - override def startBatchedWriterThread(): Unit = {} + override def startBatchedWriterThread(): Thread = writerThread override def flushRecords(): Unit = { buffer.append(walWriteQueue.take()) @@ -384,11 +452,10 @@ abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTest import WriteAheadLogSuite._ - test("records get added to a queue") { + test("BatchedWriteAheadLog - records get added to a queue") { val numSuccess = new AtomicInteger() val numFail = new AtomicInteger() - val wal = - new MockBatchedWriteAheadLog(createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching)) + val wal = new MockBatchedWriteAheadLog(mmock[FileBasedWriteAheadLog]) def getNumSuccess(): Int = numSuccess.get() def getNumFail(): Int = numFail.get() @@ -438,7 +505,7 @@ abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTest assert(waitUntilTrue(getNumSuccess, 4)) } - test("parent WriteAheadLog writes aggregated entries with the timestamp of last entry") { + test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { val parentWAL = mmock[FileBasedWriteAheadLog] val wal = new BatchedWriteAheadLog(parentWAL) @@ -471,60 +538,20 @@ abstract class BatchingWriteAheadLogRecordsTests extends CommonWriteAheadLogTest // in order of timestamp, and we need the last element. verify(parentWAL, times(1)).write(any[ByteBuffer], meq(10L)) } -} -class BatchedWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { - - override val closeFileAfterWrite: Boolean = false - override val allowBatching: Boolean = true -} - -class BatchedWithFileCloseWriteAheadLogSuite extends BatchingWriteAheadLogRecordsTests { - override val closeFileAfterWrite: Boolean = true - override val allowBatching: Boolean = true + test("BatchedWriteAheadLog - shutdown properly") { + val parentWAL = mmock[FileBasedWriteAheadLog] + val t = mmock[Thread] + val wal = new MockBatchedWriteAheadLog(parentWAL, t) - import WriteAheadLogSuite._ - test("FileBasedWriteAheadLog - close after write flag") { - // Write data with rotation using WriteAheadLog class - val numFiles = 3 - val dataToWrite = Seq.tabulate(numFiles)(_.toString) - // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed - writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, - closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) + wal.close() - // Read data manually to verify the written data - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size === numFiles) - val writtenData: Seq[String] = logFiles.flatMap { file => - val data = readDataManually[Array[Array[Byte]]](file) - data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) - } - assert(writtenData === dataToWrite) + verify(t, times(1)).interrupt() } } -class WithFileCloseWriteAheadLogSuite extends CommonWriteAheadLogTests { - override val closeFileAfterWrite: Boolean = true - override val allowBatching: Boolean = false - - import WriteAheadLogSuite._ - - test("FileBasedWriteAheadLog - close after write flag") { - // Write data with rotation using WriteAheadLog class - val numFiles = 3 - val dataToWrite = Seq.tabulate(numFiles)(_.toString) - // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed - writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100, - closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) - - // Read data manually to verify the written data - val logFiles = getLogFilesInDirectory(testDir) - assert(logFiles.size === numFiles) - val writtenData = logFiles.flatMap(file => readDataManually[String](file)) - - assert(writtenData === dataToWrite) - } -} +class BatchedWriteAheadLogWithCloseFileAfterWriteSuite + extends CloseFileAfterWriteTests(allowBatching = true, "BatchedWriteAheadLog") object WriteAheadLogSuite { @@ -541,7 +568,7 @@ object WriteAheadLogSuite { class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() - private val hadoopConf = new Configuration() + private[util] val hadoopConf = new Configuration() /** Write data to a file directly and return an array of the file segments written. */ def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { @@ -680,6 +707,17 @@ object WriteAheadLogSuite { (1 to 100).map { _.toString } } + def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { + if (allowBatching) { + logFiles.flatMap { file => + val data = readDataManually[Array[Array[Byte]]](file) + data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) + } + } else { + logFiles.flatMap { file => readDataManually[String](file)} + } + } + implicit def stringToByteBuffer(str: String): ByteBuffer = { ByteBuffer.wrap(Utils.serialize(str)) } From d1b501cc196e92bf83134df4b22d51fa9c7e967a Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 5 Nov 2015 16:49:01 -0800 Subject: [PATCH 14/26] batch-wal-writes --- .../scheduler/ReceivedBlockTracker.scala | 13 ++- .../streaming/scheduler/ReceiverTracker.scala | 4 +- .../streaming/util/BatchedWriteAheadLog.scala | 71 +++++------- .../streaming/util/WriteAheadLogSuite.scala | 94 ++-------------- .../util/WriteAheadLogUtilsSuite.scala | 102 ++++++++++++++++++ 5 files changed, 148 insertions(+), 136 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index fbe8ed6ce86bb..3bda359e8a843 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -228,10 +228,15 @@ private[streaming] class ReceivedBlockTracker( private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { logTrace(s"Writing record: $record") - val handle = writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), - clock.getTimeMillis()) - - handle != null + try { + writeAheadLogOption.get.write(ByteBuffer.wrap(Utils.serialize(record)), + clock.getTimeMillis()) + true + } catch { + case NonFatal(e) => + logWarning(s"Exception thrown while writing record: $record to the WriteAheadLog.", e) + false + } } else { true } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index b95d23654e589..9d3f1b13cd629 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -437,10 +437,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // TODO Remove this thread pool after https://github.com/apache/spark/issues/7385 is merged private val submitJobThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool")) + ThreadUtils.newDaemonCachedThreadPool("submit-job-thread-pool")) private val walBatchingThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) + ThreadUtils.newDaemonCachedThreadPool("wal-batching-thread-pool")) override def receive: PartialFunction[Any, Unit] = { // Local messages diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index f1c74dba89754..8c4ae41adcaf6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -19,36 +19,34 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import java.util.concurrent.{LinkedBlockingQueue, TimeoutException} -import java.util.{Iterator => JIterator} +import java.util.{Iterator => JIterator, ArrayList => JList} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Await, ExecutionContext, Promise} +import scala.concurrent.{Await, Promise} import scala.concurrent.duration._ import scala.util.control.NonFatal -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.util.{Utils, ThreadUtils} /** * A wrapper for a WriteAheadLog that batches records before writing data. All other methods will * be passed on to the wrapped class. - * - * Parent exposed for testing. */ -private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog) +private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) extends WriteAheadLog with Logging { import BatchedWriteAheadLog._ // exposed for tests - protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]() + protected val walWriteQueue = new LinkedBlockingQueue[Record]() private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds // Whether the writer thread is active @volatile private var active: Boolean = true - protected val buffer = new ArrayBuffer[RecordBuffer]() + protected val buffer = new JList[Record]() private val batchedWriterThread = startBatchedWriterThread() @@ -58,16 +56,8 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe */ override def write(byteBuffer: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { val promise = Promise[WriteAheadLogRecordHandle]() - walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise)) - try { - Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread), - WAL_WRITE_STATUS_TIMEOUT.milliseconds) - } catch { - case e: TimeoutException => - logWarning(s"Write to Write Ahead Log promise timed out after " + - s"$WAL_WRITE_STATUS_TIMEOUT millis for record.") - null - } + walWriteQueue.offer(Record(byteBuffer, time, promise)) + Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds) } /** @@ -77,7 +67,7 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe * This method is handled by the parent WriteAheadLog. */ override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = { - parent.read(segment) + wrappedLog.read(segment) } /** @@ -86,7 +76,7 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe * This method is handled by the parent WriteAheadLog. */ override def readAll(): JIterator[ByteBuffer] = { - parent.readAll().asScala.flatMap(deaggregate).asJava + wrappedLog.readAll().asScala.flatMap(deaggregate).asJava } /** @@ -95,7 +85,7 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe * This method is handled by the parent WriteAheadLog. */ override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { - parent.clean(threshTime, waitForCompletion) + wrappedLog.clean(threshTime, waitForCompletion) } @@ -107,19 +97,11 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe active = false batchedWriterThread.interrupt() batchedWriterThread.join() - fulfillPromises() - parent.close() - } - - /** - * Respond to any promises that may have been left in the queue, to unblock receivers during - * shutdown. - */ - private def fulfillPromises(): Unit = { while (!walWriteQueue.isEmpty) { - val RecordBuffer(_, _, promise) = walWriteQueue.poll() - promise.success(null) + val Record(_, _, promise) = walWriteQueue.poll() + promise.failure(new SparkException("close() was called on BatchedWriteAheadLog.")) } + wrappedLog.close() } /** Start the actual log writer on a separate thread. Visible (protected) for testing. */ @@ -145,8 +127,8 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe /** Write all the records in the buffer to the write ahead log. Visible for testing. */ protected def flushRecords(): Unit = { try { - buffer.append(walWriteQueue.take()) - val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 + buffer.add(walWriteQueue.take()) + val numBatched = walWriteQueue.drainTo(buffer) + 1 logDebug(s"Received $numBatched records from queue") } catch { case _: InterruptedException => @@ -154,20 +136,21 @@ private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAhe } try { var segment: WriteAheadLogRecordHandle = null - if (buffer.length > 0) { - logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") + if (buffer.size() > 0) { + logDebug(s"Batched ${buffer.size()} records for Write Ahead Log write") // we take the latest record for the time to ensure that we don't clean up files earlier // than the expiration date of the records - val time = buffer.last.time - segment = parent.write(aggregate(buffer), time) + val time = buffer.get(buff).time + segment = wrappedLog.write(aggregate(buffer), time) } buffer.foreach(_.promise.success(segment)) } catch { case NonFatal(e) => logWarning(s"Batch WAL Writer failed to write $buffer", e) - buffer.foreach(_.promise.success(null)) + buffer.foreach(_.promise.failure(e)) + } finally { + buffer.clear() } - buffer.clear() } } @@ -178,8 +161,8 @@ private[streaming] object BatchedWriteAheadLog { * with the timestamp for the write request of the record, and the promise that will block the * write request, while a separate thread is actually performing the write. */ - private[util] case class RecordBuffer( - record: ByteBuffer, + private[util] case class Record( + data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle]) @@ -191,9 +174,9 @@ private[streaming] object BatchedWriteAheadLog { } /** Aggregate multiple serialized ReceivedBlockTrackerLogEvents in a single ByteBuffer. */ - def aggregate(records: Seq[RecordBuffer]): ByteBuffer = { + def aggregate(records: Seq[Record]): ByteBuffer = { ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]]( - records.map(recordBuffer => getByteArray(recordBuffer.record)).toArray)) + records.map(record => getByteArray(record.data)).toArray)) } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 5ac9e0604b648..5da3b80865c19 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -44,84 +44,6 @@ import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{ThreadUtils, ManualClock, Utils} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -class WriteAheadLogUtilsSuite extends SparkFunSuite { - import WriteAheadLogSuite._ - - val logDir = Utils.createTempDir().getAbsolutePath() - - def assertDriverLogClass[T <: WriteAheadLog: ClassTag]( - conf: SparkConf, - isBatched: Boolean = false): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) - if (isBatched) { - assert(log.isInstanceOf[BatchedWriteAheadLog]) - val parentLog = log.asInstanceOf[BatchedWriteAheadLog].parent - assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass) - } else { - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - } - log - } - - def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = { - val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) - assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) - log - } - - test("WriteAheadLogUtils - log selection and creation") { - - val emptyConf = new SparkConf() // no log configuration - assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) - assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) - - // Verify setting driver WAL class - val conf1 = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[MockWriteAheadLog0](conf1) - assertReceiverLogClass[FileBasedWriteAheadLog](conf1) - - // Verify setting receiver WAL class - val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) - assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) - - // Verify setting receiver WAL class with 1-arg constructor - val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog1].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) - - // Verify failure setting receiver WAL class with 2-arg constructor - intercept[SparkException] { - val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog2].getName()) - assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) - } - } - - test("WriteAheadLogUtils - wrap WriteAheadLog in BatchedWriteAheadLog when batching is enabled") { - def getBatchedSparkConf: SparkConf = - new SparkConf().set("spark.streaming.driver.writeAheadLog.allowBatching", "true") - - val emptyConf = getBatchedSparkConf - assertDriverLogClass[FileBasedWriteAheadLog](emptyConf, isBatched = true) - assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) - - // Verify setting driver WAL class - val conf1 = getBatchedSparkConf.set("spark.streaming.driver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[MockWriteAheadLog0](conf1, isBatched = true) - assertReceiverLogClass[FileBasedWriteAheadLog](conf1) - - // Verify receivers are not wrapped - val receiverWALConf = getBatchedSparkConf.set("spark.streaming.receiver.writeAheadLog.class", - classOf[MockWriteAheadLog0].getName()) - assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf, isBatched = true) - assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) - } -} - /** Common tests for WriteAheadLogs that we would like to test with different configurations. */ abstract class CommonWriteAheadLogTests( allowBatching: Boolean, @@ -131,11 +53,11 @@ abstract class CommonWriteAheadLogTests( import WriteAheadLogSuite._ - val hadoopConf = new Configuration() - var tempDir: File = null - var testDir: String = null - var testFile: String = null - var writeAheadLog: WriteAheadLog = null + protected val hadoopConf = new Configuration() + protected var tempDir: File = null + protected var testDir: String = null + protected var testFile: String = null + protected var writeAheadLog: WriteAheadLog = null protected def testPrefix = if (testTag != "") testTag + " - " else testTag before { @@ -398,7 +320,7 @@ class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite { BatchCleanupEvent(Nil) ) - val buffers = events.map(e => RecordBuffer(ByteBuffer.wrap(Utils.serialize(e)), 0L, null)) + val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null)) val batched = BatchedWriteAheadLog.aggregate(buffers) val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer => Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array())) @@ -426,9 +348,9 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( walWriteQueue.drainTo(buffer.asJava) } - def mockWrite(successful: Boolean): Seq[RecordBuffer] = { + def mockWrite(successful: Boolean): Seq[Record] = { val records = buffer.toSeq - buffer.foreach { case RecordBuffer(byteBuffer, time, promise) => + buffer.foreach { case Record(byteBuffer, time, promise) => if (successful) promise.success(mmock[WriteAheadLogRecordHandle]) else promise.success(null) } buffer.clear() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala new file mode 100644 index 0000000000000..29cf6b554840f --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.util + +import scala.reflect.ClassTag + +import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} +import org.apache.spark.util.Utils + +class WriteAheadLogUtilsSuite extends SparkFunSuite { + import WriteAheadLogSuite._ + + private val logDir = Utils.createTempDir().getAbsolutePath() + + def assertDriverLogClass[T <: WriteAheadLog: ClassTag]( + conf: SparkConf, + isBatched: Boolean = false): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForDriver(conf, logDir, hadoopConf) + if (isBatched) { + assert(log.isInstanceOf[BatchedWriteAheadLog]) + val parentLog = log.asInstanceOf[BatchedWriteAheadLog].wrappedLog + assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass) + } else { + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + } + log + } + + def assertReceiverLogClass[T <: WriteAheadLog: ClassTag](conf: SparkConf): WriteAheadLog = { + val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf) + assert(log.getClass === implicitly[ClassTag[T]].runtimeClass) + log + } + + test("log selection and creation") { + + val emptyConf = new SparkConf() // no log configuration + assertDriverLogClass[FileBasedWriteAheadLog](emptyConf) + assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf) + + // Verify setting driver WAL class + val driverWALConf = new SparkConf().set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](driverWALConf) + assertReceiverLogClass[FileBasedWriteAheadLog](driverWALConf) + + // Verify setting receiver WAL class + val receiverWALConf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + + // Verify setting receiver WAL class with 1-arg constructor + val receiverWALConf2 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog1].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf2) + + // Verify failure setting receiver WAL class with 2-arg constructor + intercept[SparkException] { + val receiverWALConf3 = new SparkConf().set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog2].getName()) + assertReceiverLogClass[MockWriteAheadLog1](receiverWALConf3) + } + } + + test("wrap WriteAheadLog in BatchedWriteAheadLog when batching is enabled") { + def getBatchedSparkConf: SparkConf = + new SparkConf().set("spark.streaming.driver.writeAheadLog.allowBatching", "true") + + val justBatchingConf = getBatchedSparkConf + assertDriverLogClass[FileBasedWriteAheadLog](justBatchingConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](justBatchingConf) + + // Verify setting driver WAL class + val driverWALConf = getBatchedSparkConf.set("spark.streaming.driver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[MockWriteAheadLog0](driverWALConf, isBatched = true) + assertReceiverLogClass[FileBasedWriteAheadLog](driverWALConf) + + // Verify receivers are not wrapped + val receiverWALConf = getBatchedSparkConf.set("spark.streaming.receiver.writeAheadLog.class", + classOf[MockWriteAheadLog0].getName()) + assertDriverLogClass[FileBasedWriteAheadLog](receiverWALConf, isBatched = true) + assertReceiverLogClass[MockWriteAheadLog0](receiverWALConf) + } +} + From ec6d8da236c6a742ae523a81ebe4754013dc4a59 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Thu, 5 Nov 2015 17:32:39 -0800 Subject: [PATCH 15/26] save changes before trying something dangerous --- .../streaming/util/BatchedWriteAheadLog.scala | 61 +++++++++++-------- 1 file changed, 34 insertions(+), 27 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 8c4ae41adcaf6..82ed0aa7905fb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -18,8 +18,8 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer -import java.util.concurrent.{LinkedBlockingQueue, TimeoutException} -import java.util.{Iterator => JIterator, ArrayList => JList} +import java.util.concurrent.LinkedBlockingQueue +import java.util.{Iterator => JIterator} import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -28,13 +28,24 @@ import scala.concurrent.duration._ import scala.util.control.NonFatal import org.apache.spark.{SparkException, Logging} -import org.apache.spark.util.{Utils, ThreadUtils} +import org.apache.spark.util.Utils /** - * A wrapper for a WriteAheadLog that batches records before writing data. All other methods will - * be passed on to the wrapped class. + * A wrapper for a WriteAheadLog that batches records before writing data. Handles aggregation + * during writes, and de-aggregation in the `readAll` method. The end consumer has to handle + * de-aggregation after the `read` method. In addition, the `WriteAheadLogRecordHandle` returned + * after the write will contain the batch of records rather than individual records. + * + * When writing a batch of records, the `time` passed to the `wrappedLog` will be the timestamp + * of the latest record in the batch. This is very important in achieving correctness. Consider the + * following example: + * We receive records with timestamps 1, 3, 5, 7. We use "log-1" as the filename. Once we receive + * a clean up request for timestamp 3, we would clean up the file "log-1", and lose data regarding + * 5 and 7. + * + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. */ -private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) +private[util] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) extends WriteAheadLog with Logging { import BatchedWriteAheadLog._ @@ -46,7 +57,7 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) // Whether the writer thread is active @volatile private var active: Boolean = true - protected val buffer = new JList[Record]() + private val buffer = new ArrayBuffer[Record]() private val batchedWriterThread = startBatchedWriterThread() @@ -71,9 +82,8 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) } /** - * Read all the existing logs from the log directory. - * - * This method is handled by the parent WriteAheadLog. + * Read all the existing logs from the log directory. The output of the wrapped WriteAheadLog + * will be de-aggregated. */ override def readAll(): JIterator[ByteBuffer] = { wrappedLog.readAll().asScala.flatMap(deaggregate).asJava @@ -90,7 +100,7 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) /** - * Stop the batched writer thread, fulfill promises with failures and close parent writer. + * Stop the batched writer thread, fulfill promises with failures and close the wrapped WAL. */ override def close(): Unit = { logInfo("BatchedWriteAheadLog shutting down.") @@ -104,8 +114,8 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) wrappedLog.close() } - /** Start the actual log writer on a separate thread. Visible (protected) for testing. */ - protected def startBatchedWriterThread(): Thread = { + /** Start the actual log writer on a separate thread. */ + private def startBatchedWriterThread(): Thread = { val thread = new Thread(new Runnable { override def run(): Unit = { while (active) { @@ -124,11 +134,11 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) thread } - /** Write all the records in the buffer to the write ahead log. Visible for testing. */ - protected def flushRecords(): Unit = { + /** Write all the records in the buffer to the write ahead log. */ + private def flushRecords(): Unit = { try { - buffer.add(walWriteQueue.take()) - val numBatched = walWriteQueue.drainTo(buffer) + 1 + buffer.append(walWriteQueue.take()) + val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1 logDebug(s"Received $numBatched records from queue") } catch { case _: InterruptedException => @@ -136,11 +146,11 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) } try { var segment: WriteAheadLogRecordHandle = null - if (buffer.size() > 0) { - logDebug(s"Batched ${buffer.size()} records for Write Ahead Log write") - // we take the latest record for the time to ensure that we don't clean up files earlier - // than the expiration date of the records - val time = buffer.get(buff).time + if (buffer.length > 0) { + logDebug(s"Batched ${buffer.length} records for Write Ahead Log write") + // We take the latest record for the timestamp. Please refer to the class Javadoc for + // detailed explanation + val time = buffer.last.time segment = wrappedLog.write(aggregate(buffer), time) } buffer.foreach(_.promise.success(segment)) @@ -155,16 +165,13 @@ private[streaming] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) } /** Static methods for aggregating and de-aggregating records. */ -private[streaming] object BatchedWriteAheadLog { +private[util] object BatchedWriteAheadLog { /** * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled * with the timestamp for the write request of the record, and the promise that will block the * write request, while a separate thread is actually performing the write. */ - private[util] case class Record( - data: ByteBuffer, - time: Long, - promise: Promise[WriteAheadLogRecordHandle]) + case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle]) /** Copies the byte array of a ByteBuffer. */ private def getByteArray(buffer: ByteBuffer): Array[Byte] = { From f55469857ef4dc94ddf83ce29c89f473f5ba0c5b Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 6 Nov 2015 00:10:04 -0800 Subject: [PATCH 16/26] made updates --- .../streaming/util/BatchedWriteAheadLog.scala | 12 +- .../streaming/ReceivedBlockTrackerSuite.scala | 31 +- .../streaming/util/WriteAheadLogSuite.scala | 407 +++++------------- .../util/WriteAheadLogUtilsSuite.scala | 185 +++++++- 4 files changed, 318 insertions(+), 317 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 82ed0aa7905fb..372b1ea1b4831 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -45,15 +45,13 @@ import org.apache.spark.util.Utils * * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. */ -private[util] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog) extends WriteAheadLog with Logging { import BatchedWriteAheadLog._ // exposed for tests - protected val walWriteQueue = new LinkedBlockingQueue[Record]() - - private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds + private val walWriteQueue = new LinkedBlockingQueue[Record]() // Whether the writer thread is active @volatile private var active: Boolean = true @@ -162,10 +160,16 @@ private[util] class BatchedWriteAheadLog(wrappedLog: WriteAheadLog) buffer.clear() } } + + /** Method for querying the queue length during tests. */ + private[util] def getQueueLength(): Int = walWriteQueue.size() } /** Static methods for aggregating and de-aggregating records. */ private[util] object BatchedWriteAheadLog { + + val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds + /** * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled * with the timestamp for the write request of the record, and the promise that will block the diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 0d47d85ba0625..f793a12843b2f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -18,16 +18,13 @@ package org.apache.spark.streaming import java.io.File -import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Future, ExecutionContext} import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} import scala.util.Random import org.apache.hadoop.conf.Configuration -import org.scalatest.mock.MockitoSugar import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ @@ -37,7 +34,7 @@ import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.util.{WriteAheadLogUtils, FileBasedWriteAheadLogReader} import org.apache.spark.streaming.util.WriteAheadLogSuite._ -import org.apache.spark.util._ +import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} class ReceivedBlockTrackerSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { @@ -60,25 +57,19 @@ class ReceivedBlockTrackerSuite Utils.deleteRecursively(checkpointDirectory) } - // Generate and add blocks to the given tracker - def addBlockInfos( - tracker: ReceivedBlockTracker, - blockInfos: Seq[ReceivedBlockInfo] = generateBlockInfos()): Seq[ReceivedBlockInfo] = { - blockInfos.map(tracker.addBlock) - blockInfos - } - test("block addition, and block to batch allocation") { val receivedBlockTracker = createTracker(setCheckpointDir = false) receivedBlockTracker.isWriteAheadLogEnabled should be (false) // should be disable by default receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty - val blockInfos = addBlockInfos(receivedBlockTracker) + val blockInfos = generateBlockInfos() + blockInfos.map(receivedBlockTracker.addBlock) // Verify added blocks are unallocated blocks receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos receivedBlockTracker.hasUnallocatedReceivedBlocks should be (true) + // Allocate the blocks to a batch and verify that all of them have been allocated receivedBlockTracker.allocateBlocksToBatch(1) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos @@ -96,7 +87,7 @@ class ReceivedBlockTrackerSuite receivedBlockTracker.allocateBlocksToBatch(1) receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos - addBlockInfos(receivedBlockTracker, blockInfos) + blockInfos.map(receivedBlockTracker.addBlock) receivedBlockTracker.allocateBlocksToBatch(2) receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos @@ -112,6 +103,13 @@ class ReceivedBlockTrackerSuite manualClock.advance(timeIncrementMillis) } + // Generate and add blocks to the given tracker + def addBlockInfos(tracker: ReceivedBlockTracker): Seq[ReceivedBlockInfo] = { + val blockInfos = generateBlockInfos() + blockInfos.map(tracker.addBlock) + blockInfos + } + // Print the data present in the log ahead files in the log directory def printLogFiles(message: String) { val fileContents = getWriteAheadLogFiles().map { file => @@ -151,6 +149,7 @@ class ReceivedBlockTrackerSuite block.isBlockIdValid() should be (false) } + // Allocate blocks to batch and verify whether the unallocated blocks got allocated val batchTime1 = manualClock.getTimeMillis() tracker2.allocateBlocksToBatch(batchTime1) @@ -163,6 +162,7 @@ class ReceivedBlockTrackerSuite val blockInfos2 = addBlockInfos(tracker2) tracker2.allocateBlocksToBatch(batchTime2) tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + // Verify whether log has correct contents val expectedWrittenData2 = expectedWrittenData1 ++ Seq(createBatchAllocation(batchTime1, blockInfos1)) ++ @@ -180,7 +180,8 @@ class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - tracker3.cleanupOldBatches(batchTime2, true) + tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true) + // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty getWrittenLogData(getWriteAheadLogFiles().last) should contain(createBatchCleanup(batchTime1)) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 5da3b80865c19..e2dc22c0dfada 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -18,31 +18,28 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer -import java.util import java.util.concurrent.atomic.AtomicInteger -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{Await, Promise, Future, ExecutionContext} +import scala.concurrent._ import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} -import scala.reflect.ClassTag +import scala.util.{Failure, Success} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.mockito.Matchers._ import org.mockito.Matchers.{eq => meq} +import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ -import org.scalatest.BeforeAndAfter -import org.scalatest.mock.MockitoSugar.{mock => mmock} +import org.scalatest.{PrivateMethodTester, 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.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} /** Common tests for WriteAheadLogs that we would like to test with different configurations. */ abstract class CommonWriteAheadLogTests( @@ -74,7 +71,7 @@ abstract class CommonWriteAheadLogTests( Utils.deleteRecursively(tempDir) } - test(testPrefix + "read rotating logs") { + test(testPrefix + "read all logs") { // Write data manually for testing reading through WriteAheadLog val writtenData = (1 to 10).map { i => val data = generateRandomData() @@ -92,7 +89,7 @@ abstract class CommonWriteAheadLogTests( assert(readData === writtenData) } - test(testPrefix + "write rotating logs") { + test(testPrefix + "write logs") { // Write data with rotation using WriteAheadLog class val dataToWrite = generateRandomData() writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite = closeFileAfterWrite, @@ -108,7 +105,7 @@ abstract class CommonWriteAheadLogTests( test(testPrefix + "recover past logs when creating new manager") { // Write data with manager, recover with new manager and verify val dataToWrite = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite) + writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite, allowBatching) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) @@ -127,8 +124,8 @@ abstract class CommonWriteAheadLogTests( // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() - writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false, - closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) + writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite, + allowBatching, manualClock, closeLog = false) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) @@ -147,8 +144,8 @@ abstract class CommonWriteAheadLogTests( // Generate a set of log files val manualClock = new ManualClock val dataToWrite1 = generateRandomData() - writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock, - closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) + writeDataUsingWriteAheadLog(testDir, dataToWrite1, closeFileAfterWrite, allowBatching, + manualClock) val logFiles1 = getLogFilesInDirectory(testDir) assert(logFiles1.size > 1) @@ -156,8 +153,8 @@ abstract class CommonWriteAheadLogTests( // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock, - closeFileAfterWrite = closeFileAfterWrite, allowBatching = allowBatching) + writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching , + manualClock) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) @@ -309,11 +306,30 @@ abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String) class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog") -/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */ -class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite { +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( + allowBatching = true, + closeFileAfterWrite = false, + "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester { + import BatchedWriteAheadLog._ + import WriteAheadLogSuite._ + + private var fileBasedWAL: FileBasedWriteAheadLog = _ + private var walBatchingThreadPool: ExecutionContextExecutorService = _ + + override def beforeEach(): Unit = { + fileBasedWAL = mock[FileBasedWriteAheadLog] + walBatchingThreadPool = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool")) + } + + override def afterEach(): Unit = { + if (walBatchingThreadPool != null) { + walBatchingThreadPool.shutdownNow() + } + } - test("serializing and deserializing batched records") { + test("BatchedWriteAheadLog - serializing and deserializing batched records") { val events = Seq( BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)), BatchAllocationEvent(null, null), @@ -327,71 +343,58 @@ class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite { assert(deaggregate.toSeq === events) } -} - -class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( - allowBatching = true, - closeFileAfterWrite = false, - "BatchedWriteAheadLog") { - - import BatchedWriteAheadLog._ - // Class that will help us test batching. - private class MockBatchedWriteAheadLog( - parent: WriteAheadLog, - writerThread: Thread = mmock[Thread]) extends BatchedWriteAheadLog(parent) { + test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") { + when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!")) + // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes + val wal = new BatchedWriteAheadLog(fileBasedWAL) - override def startBatchedWriterThread(): Thread = writerThread - - override def flushRecords(): Unit = { - buffer.append(walWriteQueue.take()) - walWriteQueue.drainTo(buffer.asJava) - } - - def mockWrite(successful: Boolean): Seq[Record] = { - val records = buffer.toSeq - buffer.foreach { case Record(byteBuffer, time, promise) => - if (successful) promise.success(mmock[WriteAheadLogRecordHandle]) else promise.success(null) - } - buffer.clear() - records + intercept[RuntimeException] { + val buffer = mock[ByteBuffer] + wal.write(buffer, 2L) } - - def getQueueLength(): Int = walWriteQueue.size() } - - private def waitUntilTrue(f: () => Int, value: Int): Boolean = { - val timeOut = 2000 - val start = System.currentTimeMillis() - var result = false - while (!result && (System.currentTimeMillis() - start) < timeOut) { - Thread.sleep(50) - result = f() == value - } - result + // we make the write requests in separate threads so that we don't block the test thread + private def eventFuture( + wal: WriteAheadLog, + event: String, + time: Long, + numSuccess: AtomicInteger = null, + numFail: AtomicInteger = null): Unit = { + val f = Future(wal.write(event, time))(walBatchingThreadPool) + f.onComplete { + case Success(v) => if (numSuccess != null) numSuccess.incrementAndGet() + case Failure(v) => if (numFail != null) numFail.incrementAndGet() + }(walBatchingThreadPool) } - import WriteAheadLogSuite._ + /** + * 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] = { + // we would like to block the write so that we can queue requests + val promise = Promise[Any]() + when(wal.write(any[ByteBuffer], any[Long])).thenAnswer( + new Answer[FileBasedWriteAheadLogSegment] { + override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = { + Await.ready(promise.future, 4.seconds) + mock[FileBasedWriteAheadLogSegment] + } + } + ) + promise + } test("BatchedWriteAheadLog - records get added to a queue") { val numSuccess = new AtomicInteger() val numFail = new AtomicInteger() - val wal = new MockBatchedWriteAheadLog(mmock[FileBasedWriteAheadLog]) - - def getNumSuccess(): Int = numSuccess.get() - def getNumFail(): Int = numFail.get() + val wal = new BatchedWriteAheadLog(fileBasedWAL) - val walBatchingThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool")) - - def eventFuture(event: String, time: Long): Unit = { - val f = Future(wal.write(event, time))(walBatchingThreadPool) - f.onSuccess { case v => - if (v != null) numSuccess.incrementAndGet() else numFail.incrementAndGet() - }(walBatchingThreadPool) - } + val promise = writeBlockingPromise(fileBasedWAL) + // make sure queue is empty initially assert(wal.getQueueLength === 0) val event1 = "hello" val event2 = "world" @@ -399,46 +402,32 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val event4 = "is" val event5 = "doge" - eventFuture(event1, 5L) - assert(waitUntilTrue(wal.getQueueLength, 1)) + eventFuture(wal, event1, 5L, numSuccess, numFail) + eventFuture(wal, event2, 10L, numSuccess, numFail) + eventFuture(wal, event3, 11L, numSuccess, numFail) + eventFuture(wal, event4, 12L, numSuccess, numFail) + eventFuture(wal, event5, 20L, numSuccess, numFail) + + eventually(Eventually.timeout(2 seconds)) { + // the first element will immediately be taken and the rest will get queued + assert(wal.getQueueLength() == 4) + } assert(numSuccess.get() === 0) assert(numFail.get() === 0) + // remove block so that the writes are made + promise.success(null) - wal.flushRecords() - wal.mockWrite(successful = false) - assert(waitUntilTrue(getNumFail, 1)) - assert(waitUntilTrue(wal.getQueueLength, 0)) - - eventFuture(event2, 10L) - eventFuture(event3, 11L) - eventFuture(event4, 12L) - assert(waitUntilTrue(wal.getQueueLength, 3)) - wal.flushRecords() - wal.mockWrite(successful = true) - assert(waitUntilTrue(wal.getQueueLength, 0)) - assert(waitUntilTrue(getNumSuccess, 3)) - - eventFuture(event5, 20L) - assert(waitUntilTrue(wal.getQueueLength, 1)) - wal.flushRecords() - wal.mockWrite(successful = true) - - assert(waitUntilTrue(wal.getQueueLength, 0)) - assert(waitUntilTrue(getNumSuccess, 4)) + eventually(Eventually.timeout(2 seconds)) { + assert(wal.getQueueLength() == 0) + assert(numSuccess.get() === 5) + assert(numFail.get() == 0) + } } test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { - val parentWAL = mmock[FileBasedWriteAheadLog] - val wal = new BatchedWriteAheadLog(parentWAL) - - val promise = Promise[Boolean]() - when(parentWAL.write(any[ByteBuffer], - any[Long])).thenAnswer(new Answer[FileBasedWriteAheadLogSegment] { - override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = { - Await.ready(promise.future, 2.seconds) - mmock[FileBasedWriteAheadLogSegment] - } - }) + val wal = new BatchedWriteAheadLog(fileBasedWAL) + // block the write so that we can batch some records + val promise = writeBlockingPromise(fileBasedWAL) val event1 = "hello" val event2 = "world" @@ -446,205 +435,29 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val event4 = "is" val event5 = "doge" - wal.write(event1, 3L) // 3 will automatically be flushed for the first write - wal.write(event2, 5L) - wal.write(event3, 8L) - wal.write(event4, 12L) - wal.write(event5, 10L) // rest of the records will be batched while it takes 3 to get written + eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write + // rest of the records will be batched while it takes 3 to get written + eventFuture(wal, event2, 5L) + eventFuture(wal, event3, 8L) + eventFuture(wal, event4, 12L) + eventFuture(wal, event5, 10L) promise.success(true) - verify(parentWAL, times(1)).write(any[ByteBuffer], meq(3L)) + verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L)) - Thread.sleep(100) - // 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. - verify(parentWAL, times(1)).write(any[ByteBuffer], meq(10L)) + eventually(Eventually.timeout(1 second)) { + // 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. + verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(10L)) + } } test("BatchedWriteAheadLog - shutdown properly") { - val parentWAL = mmock[FileBasedWriteAheadLog] - val t = mmock[Thread] - val wal = new MockBatchedWriteAheadLog(parentWAL, t) - + val wal = new BatchedWriteAheadLog(fileBasedWAL) wal.close() - - verify(t, times(1)).interrupt() + verify(fileBasedWAL, times(1)).close() } } class BatchedWriteAheadLogWithCloseFileAfterWriteSuite extends CloseFileAfterWriteTests(allowBatching = true, "BatchedWriteAheadLog") - -object WriteAheadLogSuite { - - class MockWriteAheadLog0() extends WriteAheadLog { - override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } - override def read(handle: WriteAheadLogRecordHandle): ByteBuffer = { null } - override def readAll(): util.Iterator[ByteBuffer] = { null } - override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { } - override def close(): Unit = { } - } - - class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() - - class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() - - - private[util] val hadoopConf = new Configuration() - - /** Write data to a file directly and return an array of the file segments written. */ - def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { - val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() - val writer = HdfsUtils.getOutputStream(file, hadoopConf) - data.foreach { item => - val offset = writer.getPos - val bytes = Utils.serialize(item) - writer.writeInt(bytes.size) - writer.write(bytes) - segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) - } - writer.close() - segments - } - - /** - * Write data to a file using the writer class and return an array of the file segments written. - */ - def writeDataUsingWriter( - filePath: String, - data: Seq[String] - ): Seq[FileBasedWriteAheadLogSegment] = { - val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) - val segments = data.map { - item => writer.write(item) - } - writer.close() - segments - } - - /** Write data to rotating files in log directory using the WriteAheadLog class. */ - def writeDataUsingWriteAheadLog( - logDirectory: String, - data: Seq[String], - manualClock: ManualClock = new ManualClock, - closeLog: Boolean = true, - clockAdvanceTime: Int = 500, - closeFileAfterWrite: Boolean = false, - allowBatching: Boolean = false): WriteAheadLog = { - if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) - val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) - - // Ensure that 500 does not get sorted after 2000, so put a high base value. - data.foreach { item => - manualClock.advance(clockAdvanceTime) - wal.write(item, manualClock.getTimeMillis()) - } - if (closeLog) wal.close() - wal - } - - /** Read data from a segments of a log file directly and return the list of byte buffers. */ - def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = { - segments.map { segment => - val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) - try { - reader.seek(segment.offset) - val bytes = new Array[Byte](segment.length) - reader.readInt() - reader.readFully(bytes) - val data = Utils.deserialize[String](bytes) - reader.close() - data - } finally { - reader.close() - } - } - } - - /** Read all the data from a log file directly and return the list of byte buffers. */ - def readDataManually[T](file: String): Seq[T] = { - val reader = HdfsUtils.getInputStream(file, hadoopConf) - val buffer = new ArrayBuffer[T] - try { - while (true) { - // Read till EOF is thrown - val length = reader.readInt() - val bytes = new Array[Byte](length) - reader.read(bytes) - buffer += Utils.deserialize[T](bytes) - } - } catch { - case ex: EOFException => - } finally { - reader.close() - } - buffer - } - - /** Read all the data from a log file using reader class and return the list of byte buffers. */ - def readDataUsingReader(file: String): Seq[String] = { - val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) - val readData = reader.toList.map(byteBufferToString) - reader.close() - readData - } - - /** Read all the data in the log file in a directory using the WriteAheadLog class. */ - def readDataUsingWriteAheadLog( - logDirectory: String, - closeFileAfterWrite: Boolean, - allowBatching: Boolean): Seq[String] = { - val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) - val data = wal.readAll().asScala.map(byteBufferToString).toSeq - wal.close() - data - } - - /** Get the log files in a direction */ - def getLogFilesInDirectory(directory: String): Seq[String] = { - val logDirectoryPath = new Path(directory) - val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - - if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { - fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { - _.getName().split("-")(1).toLong - }.map { - _.toString.stripPrefix("file:") - } - } else { - Seq.empty - } - } - - def createWriteAheadLog( - logDirectory: String, - closeFileAfterWrite: Boolean, - allowBatching: Boolean): WriteAheadLog = { - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite) - if (allowBatching) new BatchedWriteAheadLog(wal) else wal - } - - def generateRandomData(): Seq[String] = { - (1 to 100).map { _.toString } - } - - def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { - if (allowBatching) { - logFiles.flatMap { file => - val data = readDataManually[Array[Array[Byte]]](file) - data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) - } - } else { - logFiles.flatMap { file => readDataManually[String](file)} - } - } - - implicit def stringToByteBuffer(str: String): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize(str)) - } - - implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { - Utils.deserialize[String](byteBuffer.array) - } -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala index 29cf6b554840f..4e9e92aac328c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala @@ -17,15 +17,27 @@ package org.apache.spark.streaming.util +import java.io.EOFException +import java.nio.ByteBuffer +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.language.{implicitConversions, postfixOps} import scala.reflect.ClassTag +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.PrivateMethodTester + import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ManualClock, Utils} class WriteAheadLogUtilsSuite extends SparkFunSuite { import WriteAheadLogSuite._ private val logDir = Utils.createTempDir().getAbsolutePath() + private val hadoopConf = new Configuration() def assertDriverLogClass[T <: WriteAheadLog: ClassTag]( conf: SparkConf, @@ -100,3 +112,174 @@ class WriteAheadLogUtilsSuite extends SparkFunSuite { } } +object WriteAheadLogSuite { + + class MockWriteAheadLog0() extends WriteAheadLog { + override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } + override def read(handle: WriteAheadLogRecordHandle): ByteBuffer = { null } + override def readAll(): util.Iterator[ByteBuffer] = { null } + override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = { } + override def close(): Unit = { } + } + + class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() + + class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() + + private val hadoopConf = new Configuration() + + /** Write data to a file directly and return an array of the file segments written. */ + def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { + val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() + val writer = HdfsUtils.getOutputStream(file, hadoopConf) + data.foreach { item => + val offset = writer.getPos + val bytes = Utils.serialize(item) + writer.writeInt(bytes.size) + writer.write(bytes) + segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) + } + writer.close() + segments + } + + /** + * Write data to a file using the writer class and return an array of the file segments written. + */ + def writeDataUsingWriter( + filePath: String, + data: Seq[String]): Seq[FileBasedWriteAheadLogSegment] = { + val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) + val segments = data.map { + item => writer.write(item) + } + writer.close() + segments + } + + /** Write data to rotating files in log directory using the WriteAheadLog class. */ + def writeDataUsingWriteAheadLog( + logDirectory: String, + data: Seq[String], + closeFileAfterWrite: Boolean, + allowBatching: Boolean, + manualClock: ManualClock = new ManualClock, + closeLog: Boolean = true, + clockAdvanceTime: Int = 500): WriteAheadLog = { + if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) + + // Ensure that 500 does not get sorted after 2000, so put a high base value. + data.foreach { item => + manualClock.advance(clockAdvanceTime) + wal.write(item, manualClock.getTimeMillis()) + } + if (closeLog) wal.close() + wal + } + + /** Read data from a segments of a log file directly and return the list of byte buffers. */ + def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = { + segments.map { segment => + val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) + try { + reader.seek(segment.offset) + val bytes = new Array[Byte](segment.length) + reader.readInt() + reader.readFully(bytes) + val data = Utils.deserialize[String](bytes) + reader.close() + data + } finally { + reader.close() + } + } + } + + /** Read all the data from a log file directly and return the list of byte buffers. */ + def readDataManually[T](file: String): Seq[T] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + val buffer = new ArrayBuffer[T] + try { + while (true) { + // Read till EOF is thrown + val length = reader.readInt() + val bytes = new Array[Byte](length) + reader.read(bytes) + buffer += Utils.deserialize[T](bytes) + } + } catch { + case ex: EOFException => + } finally { + reader.close() + } + buffer + } + + /** Read all the data from a log file using reader class and return the list of byte buffers. */ + def readDataUsingReader(file: String): Seq[String] = { + val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) + val readData = reader.toList.map(byteBufferToString) + reader.close() + readData + } + + /** Read all the data in the log file in a directory using the WriteAheadLog class. */ + def readDataUsingWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): Seq[String] = { + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) + val data = wal.readAll().asScala.map(byteBufferToString).toSeq + wal.close() + data + } + + /** Get the log files in a direction */ + def getLogFilesInDirectory(directory: String): Seq[String] = { + val logDirectoryPath = new Path(directory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { + _.getName().split("-")(1).toLong + }.map { + _.toString.stripPrefix("file:") + } + } else { + Seq.empty + } + } + + def createWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): WriteAheadLog = { + val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, + closeFileAfterWrite) + if (allowBatching) new BatchedWriteAheadLog(wal) else wal + } + + def generateRandomData(): Seq[String] = { + (1 to 100).map { _.toString } + } + + def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { + if (allowBatching) { + logFiles.flatMap { file => + val data = readDataManually[Array[Array[Byte]]](file) + data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) + } + } else { + logFiles.flatMap { file => readDataManually[String](file)} + } + } + + implicit def stringToByteBuffer(str: String): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(str)) + } + + implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { + Utils.deserialize[String](byteBuffer.array) + } +} From f3f79dda27cb35b22d00ff5709571ec26f1cab6f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 6 Nov 2015 00:18:01 -0800 Subject: [PATCH 17/26] minor2 --- .../streaming/scheduler/ReceivedBlockTracker.scala | 2 +- .../spark/streaming/util/WriteAheadLogSuite.scala | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 3bda359e8a843..500dc70c98506 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -225,7 +225,7 @@ private[streaming] class ReceivedBlockTracker( } /** Write an update to the tracker to the write ahead log */ - private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { + private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = { if (isWriteAheadLogEnabled) { logTrace(s"Writing record: $record") try { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index e2dc22c0dfada..71fe34d44299d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -102,7 +102,7 @@ abstract class CommonWriteAheadLogTests( assert(writtenData === dataToWrite) } - test(testPrefix + "recover past logs when creating new manager") { + test(testPrefix + "read all logs after write") { // Write data with manager, recover with new manager and verify val dataToWrite = generateRandomData() writeDataUsingWriteAheadLog(testDir, dataToWrite, closeFileAfterWrite, allowBatching) @@ -315,10 +315,12 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( import WriteAheadLogSuite._ private var fileBasedWAL: FileBasedWriteAheadLog = _ + private var walHandle: FileBasedWriteAheadLogSegment = _ private var walBatchingThreadPool: ExecutionContextExecutorService = _ override def beforeEach(): Unit = { fileBasedWAL = mock[FileBasedWriteAheadLog] + walHandle = mock[FileBasedWriteAheadLogSegment] walBatchingThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool")) } @@ -364,7 +366,9 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( numFail: AtomicInteger = null): Unit = { val f = Future(wal.write(event, time))(walBatchingThreadPool) f.onComplete { - case Success(v) => if (numSuccess != null) numSuccess.incrementAndGet() + case Success(v) => + assert(v === walHandle) // return our mock handle after the write + if (numSuccess != null) numSuccess.incrementAndGet() case Failure(v) => if (numFail != null) numFail.incrementAndGet() }(walBatchingThreadPool) } @@ -380,7 +384,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( new Answer[FileBasedWriteAheadLogSegment] { override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = { Await.ready(promise.future, 4.seconds) - mock[FileBasedWriteAheadLogSegment] + walHandle } } ) From 84194c04d98868a083040ac0ab72ceb5e1c44ab8 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 6 Nov 2015 13:01:10 -0800 Subject: [PATCH 18/26] address I don't know the count anymore --- .../streaming/util/BatchedWriteAheadLog.scala | 44 ++++++++++++------- .../streaming/util/WriteAheadLogUtils.scala | 11 ++++- .../streaming/util/WriteAheadLogSuite.scala | 14 +++--- .../util/WriteAheadLogUtilsSuite.scala | 5 ++- 4 files changed, 48 insertions(+), 26 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 372b1ea1b4831..aea5c6bc9a7f2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -27,7 +27,7 @@ import scala.concurrent.{Await, Promise} import scala.concurrent.duration._ import scala.util.control.NonFatal -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.{SparkConf, SparkException, Logging} import org.apache.spark.util.Utils /** @@ -45,12 +45,11 @@ import org.apache.spark.util.Utils * * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. */ -private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog) +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: SparkConf) extends WriteAheadLog with Logging { import BatchedWriteAheadLog._ - // exposed for tests private val walWriteQueue = new LinkedBlockingQueue[Record]() // Whether the writer thread is active @@ -65,18 +64,30 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog) */ override def write(byteBuffer: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { val promise = Promise[WriteAheadLogRecordHandle]() - walWriteQueue.offer(Record(byteBuffer, time, promise)) - Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds) + val putSuccessfully = synchronized { + if (active) { + walWriteQueue.offer(Record(byteBuffer, time, promise)) + true + } else { + false + } + } + if (putSuccessfully) { + Await.result(promise.future, WriteAheadLogUtils.getBatchingTimeout(conf).milliseconds) + } else { + throw new SparkException("close() was called on BatchedWriteAheadLog before " + + s"write request with time $time could be fulfilled.") + } } /** - * Read a segment from an existing Write Ahead Log. The data may be aggregated, and the user - * should de-aggregate using [[BatchedWriteAheadLog.deaggregate]] - * - * This method is handled by the parent WriteAheadLog. + * This method is not supported as the resulting ByteBuffer would actually require de-aggregation. + * This method is primarily used in testing, and to ensure that it is not used in production, + * we throw an UnsupportedOperationException. */ override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = { - wrappedLog.read(segment) + throw new UnsupportedOperationException("read() is not supported for BatchedWriteAheadLog " + + "as the data may require de-aggregation.") } /** @@ -101,13 +112,16 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog) * Stop the batched writer thread, fulfill promises with failures and close the wrapped WAL. */ override def close(): Unit = { - logInfo("BatchedWriteAheadLog shutting down.") - active = false + logInfo(s"BatchedWriteAheadLog shutting down at time: ${System.currentTimeMillis()}.") + synchronized { + active = false + } batchedWriterThread.interrupt() batchedWriterThread.join() while (!walWriteQueue.isEmpty) { - val Record(_, _, promise) = walWriteQueue.poll() - promise.failure(new SparkException("close() was called on BatchedWriteAheadLog.")) + val Record(_, time, promise) = walWriteQueue.poll() + promise.failure(new SparkException("close() was called on BatchedWriteAheadLog before " + + s"write request with time $time could be fulfilled.")) } wrappedLog.close() } @@ -168,8 +182,6 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog) /** Static methods for aggregating and de-aggregating records. */ private[util] object BatchedWriteAheadLog { - val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds - /** * Wrapper class for representing the records that we will write to the WriteAheadLog. Coupled * with the timestamp for the write request of the record, and the promise that will block the diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 34d1ee66cb9e7..731a369fc92c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -39,6 +39,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { "spark.streaming.driver.writeAheadLog.rollingIntervalSecs" val DRIVER_WAL_MAX_FAILURES_CONF_KEY = "spark.streaming.driver.writeAheadLog.maxFailures" val DRIVER_WAL_BATCHING_CONF_KEY = "spark.streaming.driver.writeAheadLog.allowBatching" + val DRIVER_WAL_BATCHING_TIMEOUT_CONF_KEY = "spark.streaming.driver.writeAheadLog.batchingTimeout" val DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY = "spark.streaming.driver.writeAheadLog.closeFileAfterWrite" @@ -69,6 +70,14 @@ private[streaming] object WriteAheadLogUtils extends Logging { isDriver && conf.getBoolean(DRIVER_WAL_BATCHING_CONF_KEY, defaultValue = false) } + /** + * How long we will wait for the wrappedLog in the BatchedWriteAheadLog to write the records + * before we fail the write attempt to unblock receivers. + */ + def getBatchingTimeout(conf: SparkConf): Long = { + conf.getLong(DRIVER_WAL_BATCHING_TIMEOUT_CONF_KEY, defaultValue = 5000) + } + def shouldCloseFileAfterWrite(conf: SparkConf, isDriver: Boolean): Boolean = { if (isDriver) { conf.getBoolean(DRIVER_WAL_CLOSE_AFTER_WRITE_CONF_KEY, defaultValue = false) @@ -134,7 +143,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { shouldCloseFileAfterWrite(sparkConf, isDriver)) } if (isBatchingEnabled(sparkConf, isDriver)) { - new BatchedWriteAheadLog(wal) + new BatchedWriteAheadLog(wal, sparkConf) } else { wal } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 71fe34d44299d..aa0b2b8fcfd1e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -317,6 +317,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( private var fileBasedWAL: FileBasedWriteAheadLog = _ private var walHandle: FileBasedWriteAheadLogSegment = _ private var walBatchingThreadPool: ExecutionContextExecutorService = _ + private val sparkConf = new SparkConf() override def beforeEach(): Unit = { fileBasedWAL = mock[FileBasedWriteAheadLog] @@ -349,7 +350,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") { when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!")) // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes - val wal = new BatchedWriteAheadLog(fileBasedWAL) + val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) intercept[RuntimeException] { val buffer = mock[ByteBuffer] @@ -394,7 +395,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( test("BatchedWriteAheadLog - records get added to a queue") { val numSuccess = new AtomicInteger() val numFail = new AtomicInteger() - val wal = new BatchedWriteAheadLog(fileBasedWAL) + val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) val promise = writeBlockingPromise(fileBasedWAL) @@ -414,7 +415,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( eventually(Eventually.timeout(2 seconds)) { // the first element will immediately be taken and the rest will get queued - assert(wal.getQueueLength() == 4) + assert(wal.getQueueLength() > 0) } assert(numSuccess.get() === 0) assert(numFail.get() === 0) @@ -429,7 +430,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( } test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { - val wal = new BatchedWriteAheadLog(fileBasedWAL) + val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) // block the write so that we can batch some records val promise = writeBlockingPromise(fileBasedWAL) @@ -447,9 +448,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( eventFuture(wal, event5, 10L) promise.success(true) - verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L)) - eventually(Eventually.timeout(1 second)) { + verify(fileBasedWAL, times(1)).write(any[ByteBuffer], 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. verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(10L)) @@ -457,7 +457,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( } test("BatchedWriteAheadLog - shutdown properly") { - val wal = new BatchedWriteAheadLog(fileBasedWAL) + val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) wal.close() verify(fileBasedWAL, times(1)).close() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala index 4e9e92aac328c..95a788f52ba57 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala @@ -255,9 +255,10 @@ object WriteAheadLogSuite { logDirectory: String, closeFileAfterWrite: Boolean, allowBatching: Boolean): WriteAheadLog = { - val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1, + val sparkConf = new SparkConf + val wal = new FileBasedWriteAheadLog(sparkConf, logDirectory, hadoopConf, 1, 1, closeFileAfterWrite) - if (allowBatching) new BatchedWriteAheadLog(wal) else wal + if (allowBatching) new BatchedWriteAheadLog(wal, sparkConf) else wal } def generateRandomData(): Seq[String] = { From 21f44eaf793cfbbbafc7573583c645feab44af3f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Fri, 6 Nov 2015 13:11:25 -0800 Subject: [PATCH 19/26] add minor explanation --- .../org/apache/spark/streaming/util/BatchedWriteAheadLog.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index aea5c6bc9a7f2..c1380a040026a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -43,6 +43,9 @@ import org.apache.spark.util.Utils * a clean up request for timestamp 3, we would clean up the file "log-1", and lose data regarding * 5 and 7. * + * In addition, notice that the write method is still a blocking call. This will ensure that a + * receiver will not be able to submit multiple `AddBlock` calls, jeopardizing the ordering of data. + * * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. */ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: SparkConf) From 2c6d5154339b772a6cce66696f12d65e39c2779e Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 8 Nov 2015 12:00:07 -0800 Subject: [PATCH 20/26] added synchronized checks for shutdown --- .../streaming/scheduler/ReceiverTracker.scala | 25 ++++++++++++- .../streaming/util/WriteAheadLogSuite.scala | 37 ------------------- 2 files changed, 23 insertions(+), 39 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 9d3f1b13cd629..9a98dd5819a71 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -442,6 +442,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private val walBatchingThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("wal-batching-thread-pool")) + private var active: Boolean = true + override def receive: PartialFunction[Any, Unit] = { // Local messages case StartAllReceivers(receivers) => @@ -492,8 +494,24 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(successful) case AddBlock(receivedBlockInfo) => if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { - val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool) - f.onComplete(result => context.reply(result.get))(walBatchingThreadPool) + val f = Future { + synchronized { + if (active) { + addBlock(receivedBlockInfo) + } else { + throw new IllegalStateException("Receiver Tracker Endpoint shutdown.") + } + } + }(walBatchingThreadPool) + f.onComplete { result => + synchronized { + if (active) { + context.reply(result.get) + } else { + context.reply(new IllegalStateException("Receiver Tracker Endpoint shutdown.")) + } + } + }(walBatchingThreadPool) } else { context.reply(addBlock(receivedBlockInfo)) } @@ -607,6 +625,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def onStop(): Unit = { submitJobThreadPool.shutdownNow() + synchronized { + active = false + } walBatchingThreadPool.shutdown() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index aa0b2b8fcfd1e..1e5ff1ae628f4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -392,43 +392,6 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( promise } - test("BatchedWriteAheadLog - records get added to a queue") { - val numSuccess = new AtomicInteger() - val numFail = new AtomicInteger() - val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) - - val promise = writeBlockingPromise(fileBasedWAL) - - // make sure queue is empty initially - assert(wal.getQueueLength === 0) - val event1 = "hello" - val event2 = "world" - val event3 = "this" - val event4 = "is" - val event5 = "doge" - - eventFuture(wal, event1, 5L, numSuccess, numFail) - eventFuture(wal, event2, 10L, numSuccess, numFail) - eventFuture(wal, event3, 11L, numSuccess, numFail) - eventFuture(wal, event4, 12L, numSuccess, numFail) - eventFuture(wal, event5, 20L, numSuccess, numFail) - - eventually(Eventually.timeout(2 seconds)) { - // the first element will immediately be taken and the rest will get queued - assert(wal.getQueueLength() > 0) - } - assert(numSuccess.get() === 0) - assert(numFail.get() === 0) - // remove block so that the writes are made - promise.success(null) - - eventually(Eventually.timeout(2 seconds)) { - assert(wal.getQueueLength() == 0) - assert(numSuccess.get() === 5) - assert(numFail.get() == 0) - } - } - test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) // block the write so that we can batch some records From c659cd7cfd9ef011f16583752aea4fcd244c3e02 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 8 Nov 2015 21:03:00 -0800 Subject: [PATCH 21/26] address y --- .../streaming/scheduler/ReceiverTracker.scala | 16 +- .../streaming/util/BatchedWriteAheadLog.scala | 22 +- .../streaming/util/WriteAheadLogSuite.scala | 241 +++++++++++++++--- .../util/WriteAheadLogUtilsSuite.scala | 170 +----------- 4 files changed, 230 insertions(+), 219 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 9a98dd5819a71..207b3dc95a16e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -442,7 +442,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false private val walBatchingThreadPool = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("wal-batching-thread-pool")) - private var active: Boolean = true + @volatile private var active: Boolean = true override def receive: PartialFunction[Any, Unit] = { // Local messages @@ -495,22 +495,18 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false case AddBlock(receivedBlockInfo) => if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { val f = Future { - synchronized { - if (active) { - addBlock(receivedBlockInfo) - } else { - throw new IllegalStateException("Receiver Tracker Endpoint shutdown.") - } + if (active) { + addBlock(receivedBlockInfo) + } else { + throw new IllegalStateException("Receiver Tracker Endpoint shut down.") } }(walBatchingThreadPool) f.onComplete { result => - synchronized { if (active) { context.reply(result.get) } else { - context.reply(new IllegalStateException("Receiver Tracker Endpoint shutdown.")) + throw new IllegalStateException("Receiver Tracker Endpoint shut down.") } - } }(walBatchingThreadPool) } else { context.reply(addBlock(receivedBlockInfo)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index c1380a040026a..08bfb1517f6d5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -27,7 +27,7 @@ import scala.concurrent.{Await, Promise} import scala.concurrent.duration._ import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkException, Logging} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils /** @@ -43,8 +43,10 @@ import org.apache.spark.util.Utils * a clean up request for timestamp 3, we would clean up the file "log-1", and lose data regarding * 5 and 7. * - * In addition, notice that the write method is still a blocking call. This will ensure that a - * receiver will not be able to submit multiple `AddBlock` calls, jeopardizing the ordering of data. + * This means the caller can assume the same write semantics as any other WriteAheadLog + * implementation despite the batching in the background - when the write() returns, the data is + * written to the WAL and is durable. To take advantage of the batching, the caller can write from + * multiple threads, each of which will stay blocked until the corresponding data has been written. * * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog. */ @@ -78,7 +80,7 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp if (putSuccessfully) { Await.result(promise.future, WriteAheadLogUtils.getBatchingTimeout(conf).milliseconds) } else { - throw new SparkException("close() was called on BatchedWriteAheadLog before " + + throw new IllegalStateException("close() was called on BatchedWriteAheadLog before " + s"write request with time $time could be fulfilled.") } } @@ -123,8 +125,8 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp batchedWriterThread.join() while (!walWriteQueue.isEmpty) { val Record(_, time, promise) = walWriteQueue.poll() - promise.failure(new SparkException("close() was called on BatchedWriteAheadLog before " + - s"write request with time $time could be fulfilled.")) + promise.failure(new IllegalStateException("close() was called on BatchedWriteAheadLog " + + s"before write request with time $time could be fulfilled.")) } wrappedLog.close() } @@ -141,9 +143,9 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp logWarning("Encountered exception in Batched Writer Thread.", e) } } - logInfo("Batched WAL Writer thread exiting.") + logInfo("BatchedWriteAheadLog Writer thread exiting.") } - }, "Batched WAL Writer") + }, "BatchedWriteAheadLog Writer") thread.setDaemon(true) thread.start() thread @@ -157,7 +159,7 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp logDebug(s"Received $numBatched records from queue") } catch { case _: InterruptedException => - logWarning("Batch Write Ahead Log Writer queue interrupted.") + logWarning("BatchedWriteAheadLog Writer queue interrupted.") } try { var segment: WriteAheadLogRecordHandle = null @@ -171,7 +173,7 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp buffer.foreach(_.promise.success(segment)) } catch { case NonFatal(e) => - logWarning(s"Batch WAL Writer failed to write $buffer", e) + logWarning(s"BatchedWriteAheadLog Writer failed to write $buffer", e) buffer.foreach(_.promise.failure(e)) } finally { buffer.clear() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 1e5ff1ae628f4..fea124abe5436 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -18,8 +18,11 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer +import java.util.concurrent.ThreadPoolExecutor import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent._ import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} @@ -34,7 +37,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ -import org.scalatest.{PrivateMethodTester, BeforeAndAfterEach, BeforeAndAfter} +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfter} import org.scalatest.mock.MockitoSugar import org.apache.spark.streaming.scheduler._ @@ -309,26 +312,27 @@ class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( allowBatching = true, closeFileAfterWrite = false, - "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester { + "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with Eventually { import BatchedWriteAheadLog._ import WriteAheadLogSuite._ - private var fileBasedWAL: FileBasedWriteAheadLog = _ - private var walHandle: FileBasedWriteAheadLogSegment = _ - private var walBatchingThreadPool: ExecutionContextExecutorService = _ + private var wal: WriteAheadLog = _ + private var walHandle: WriteAheadLogRecordHandle = _ + private var walBatchingThreadPool: ThreadPoolExecutor = _ + private var walBatchingExecutionContext: ExecutionContextExecutorService = _ private val sparkConf = new SparkConf() override def beforeEach(): Unit = { - fileBasedWAL = mock[FileBasedWriteAheadLog] - walHandle = mock[FileBasedWriteAheadLogSegment] - walBatchingThreadPool = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool")) + wal = mock[WriteAheadLog] + walHandle = mock[WriteAheadLogRecordHandle] + walBatchingThreadPool = ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool") + walBatchingExecutionContext = ExecutionContext.fromExecutorService(walBatchingThreadPool) } override def afterEach(): Unit = { - if (walBatchingThreadPool != null) { - walBatchingThreadPool.shutdownNow() + if (walBatchingExecutionContext != null) { + walBatchingExecutionContext.shutdownNow() } } @@ -348,30 +352,30 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( } test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") { - when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!")) + when(wal.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!")) // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes - val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) + val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) intercept[RuntimeException] { val buffer = mock[ByteBuffer] - wal.write(buffer, 2L) + batchedWal.write(buffer, 2L) } } // we make the write requests in separate threads so that we don't block the test thread - private def eventFuture( + private def writeEventWithFuture( wal: WriteAheadLog, event: String, time: Long, numSuccess: AtomicInteger = null, numFail: AtomicInteger = null): Unit = { - val f = Future(wal.write(event, time))(walBatchingThreadPool) + val f = Future(wal.write(event, time))(walBatchingExecutionContext) f.onComplete { case Success(v) => assert(v === walHandle) // return our mock handle after the write if (numSuccess != null) numSuccess.incrementAndGet() case Failure(v) => if (numFail != null) numFail.incrementAndGet() - }(walBatchingThreadPool) + }(walBatchingExecutionContext) } /** @@ -382,8 +386,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // we would like to block the write so that we can queue requests val promise = Promise[Any]() when(wal.write(any[ByteBuffer], any[Long])).thenAnswer( - new Answer[FileBasedWriteAheadLogSegment] { - override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = { + new Answer[WriteAheadLogRecordHandle] { + override def answer(invocation: InvocationOnMock): WriteAheadLogRecordHandle = { Await.ready(promise.future, 4.seconds) walHandle } @@ -393,9 +397,9 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( } test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { - val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) + val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) // block the write so that we can batch some records - val promise = writeBlockingPromise(fileBasedWAL) + val promise = writeBlockingPromise(wal) val event1 = "hello" val event2 = "world" @@ -403,28 +407,201 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val event4 = "is" val event5 = "doge" - eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write + writeEventWithFuture(batchedWal, event1, 3L) // 3 will automatically be flushed for the first write // rest of the records will be batched while it takes 3 to get written - eventFuture(wal, event2, 5L) - eventFuture(wal, event3, 8L) - eventFuture(wal, event4, 12L) - eventFuture(wal, event5, 10L) + writeEventWithFuture(batchedWal, event2, 5L) + writeEventWithFuture(batchedWal, event3, 8L) + writeEventWithFuture(batchedWal, event4, 12L) + writeEventWithFuture(batchedWal, event5, 10L) + eventually(timeout(1 second)) { + assert(walBatchingThreadPool.getActiveCount === 5) + } promise.success(true) - eventually(Eventually.timeout(1 second)) { - verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L)) + val buffer1 = wrapArrayArrayByte(Array(event1)) + val buffer2 = wrapArrayArrayByte(Array(event2, event3, event4, event5)) + + eventually(timeout(1 second)) { + 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. - verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(10L)) + verify(wal, times(1)).write(meq(buffer2), meq(10L)) } } test("BatchedWriteAheadLog - shutdown properly") { - val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf) - wal.close() - verify(fileBasedWAL, times(1)).close() + val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) + batchedWal.close() + verify(wal, times(1)).close() + + intercept[IllegalStateException](batchedWal.write(mock[ByteBuffer], 12L)) } } class BatchedWriteAheadLogWithCloseFileAfterWriteSuite extends CloseFileAfterWriteTests(allowBatching = true, "BatchedWriteAheadLog") + +object WriteAheadLogSuite { + + private val hadoopConf = new Configuration() + + /** Write data to a file directly and return an array of the file segments written. */ + def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { + val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() + val writer = HdfsUtils.getOutputStream(file, hadoopConf) + data.foreach { item => + val offset = writer.getPos + val bytes = Utils.serialize(item) + writer.writeInt(bytes.size) + writer.write(bytes) + segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) + } + writer.close() + segments + } + + /** + * Write data to a file using the writer class and return an array of the file segments written. + */ + def writeDataUsingWriter( + filePath: String, + data: Seq[String]): Seq[FileBasedWriteAheadLogSegment] = { + val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) + val segments = data.map { + item => writer.write(item) + } + writer.close() + segments + } + + /** Write data to rotating files in log directory using the WriteAheadLog class. */ + def writeDataUsingWriteAheadLog( + logDirectory: String, + data: Seq[String], + closeFileAfterWrite: Boolean, + allowBatching: Boolean, + manualClock: ManualClock = new ManualClock, + closeLog: Boolean = true, + clockAdvanceTime: Int = 500): WriteAheadLog = { + if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) + + // Ensure that 500 does not get sorted after 2000, so put a high base value. + data.foreach { item => + manualClock.advance(clockAdvanceTime) + wal.write(item, manualClock.getTimeMillis()) + } + if (closeLog) wal.close() + wal + } + + /** Read data from a segments of a log file directly and return the list of byte buffers. */ + def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = { + segments.map { segment => + val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) + try { + reader.seek(segment.offset) + val bytes = new Array[Byte](segment.length) + reader.readInt() + reader.readFully(bytes) + val data = Utils.deserialize[String](bytes) + reader.close() + data + } finally { + reader.close() + } + } + } + + /** Read all the data from a log file directly and return the list of byte buffers. */ + def readDataManually[T](file: String): Seq[T] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + val buffer = new ArrayBuffer[T] + try { + while (true) { + // Read till EOF is thrown + val length = reader.readInt() + val bytes = new Array[Byte](length) + reader.read(bytes) + buffer += Utils.deserialize[T](bytes) + } + } catch { + case ex: EOFException => + } finally { + reader.close() + } + buffer + } + + /** Read all the data from a log file using reader class and return the list of byte buffers. */ + def readDataUsingReader(file: String): Seq[String] = { + val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) + val readData = reader.toList.map(byteBufferToString) + reader.close() + readData + } + + /** Read all the data in the log file in a directory using the WriteAheadLog class. */ + def readDataUsingWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): Seq[String] = { + val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) + val data = wal.readAll().asScala.map(byteBufferToString).toSeq + wal.close() + data + } + + /** Get the log files in a directory. */ + def getLogFilesInDirectory(directory: String): Seq[String] = { + val logDirectoryPath = new Path(directory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { + _.getName().split("-")(1).toLong + }.map { + _.toString.stripPrefix("file:") + } + } else { + Seq.empty + } + } + + def createWriteAheadLog( + logDirectory: String, + closeFileAfterWrite: Boolean, + allowBatching: Boolean): WriteAheadLog = { + val sparkConf = new SparkConf + val wal = new FileBasedWriteAheadLog(sparkConf, logDirectory, hadoopConf, 1, 1, + closeFileAfterWrite) + if (allowBatching) new BatchedWriteAheadLog(wal, sparkConf) else wal + } + + def generateRandomData(): Seq[String] = { + (1 to 100).map { _.toString } + } + + def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { + if (allowBatching) { + logFiles.flatMap { file => + val data = readDataManually[Array[Array[Byte]]](file) + data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) + } + } else { + logFiles.flatMap { file => readDataManually[String](file)} + } + } + + implicit def stringToByteBuffer(str: String): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(str)) + } + + implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { + Utils.deserialize[String](byteBuffer.array) + } + + def wrapArrayArrayByte[T](records: Array[T]): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(Utils.serialize[T]))) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala index 95a788f52ba57..9152728191ea1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala @@ -17,24 +17,18 @@ package org.apache.spark.streaming.util -import java.io.EOFException import java.nio.ByteBuffer import java.util -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.language.{implicitConversions, postfixOps} import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} -import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.util.Utils class WriteAheadLogUtilsSuite extends SparkFunSuite { - import WriteAheadLogSuite._ + import WriteAheadLogUtilsSuite._ private val logDir = Utils.createTempDir().getAbsolutePath() private val hadoopConf = new Configuration() @@ -112,7 +106,7 @@ class WriteAheadLogUtilsSuite extends SparkFunSuite { } } -object WriteAheadLogSuite { +object WriteAheadLogUtilsSuite { class MockWriteAheadLog0() extends WriteAheadLog { override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { null } @@ -125,162 +119,4 @@ object WriteAheadLogSuite { class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0() class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0() - - private val hadoopConf = new Configuration() - - /** Write data to a file directly and return an array of the file segments written. */ - def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { - val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() - val writer = HdfsUtils.getOutputStream(file, hadoopConf) - data.foreach { item => - val offset = writer.getPos - val bytes = Utils.serialize(item) - writer.writeInt(bytes.size) - writer.write(bytes) - segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) - } - writer.close() - segments - } - - /** - * Write data to a file using the writer class and return an array of the file segments written. - */ - def writeDataUsingWriter( - filePath: String, - data: Seq[String]): Seq[FileBasedWriteAheadLogSegment] = { - val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf) - val segments = data.map { - item => writer.write(item) - } - writer.close() - segments - } - - /** Write data to rotating files in log directory using the WriteAheadLog class. */ - def writeDataUsingWriteAheadLog( - logDirectory: String, - data: Seq[String], - closeFileAfterWrite: Boolean, - allowBatching: Boolean, - manualClock: ManualClock = new ManualClock, - closeLog: Boolean = true, - clockAdvanceTime: Int = 500): WriteAheadLog = { - if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000) - val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) - - // Ensure that 500 does not get sorted after 2000, so put a high base value. - data.foreach { item => - manualClock.advance(clockAdvanceTime) - wal.write(item, manualClock.getTimeMillis()) - } - if (closeLog) wal.close() - wal - } - - /** Read data from a segments of a log file directly and return the list of byte buffers. */ - def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = { - segments.map { segment => - val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) - try { - reader.seek(segment.offset) - val bytes = new Array[Byte](segment.length) - reader.readInt() - reader.readFully(bytes) - val data = Utils.deserialize[String](bytes) - reader.close() - data - } finally { - reader.close() - } - } - } - - /** Read all the data from a log file directly and return the list of byte buffers. */ - def readDataManually[T](file: String): Seq[T] = { - val reader = HdfsUtils.getInputStream(file, hadoopConf) - val buffer = new ArrayBuffer[T] - try { - while (true) { - // Read till EOF is thrown - val length = reader.readInt() - val bytes = new Array[Byte](length) - reader.read(bytes) - buffer += Utils.deserialize[T](bytes) - } - } catch { - case ex: EOFException => - } finally { - reader.close() - } - buffer - } - - /** Read all the data from a log file using reader class and return the list of byte buffers. */ - def readDataUsingReader(file: String): Seq[String] = { - val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) - val readData = reader.toList.map(byteBufferToString) - reader.close() - readData - } - - /** Read all the data in the log file in a directory using the WriteAheadLog class. */ - def readDataUsingWriteAheadLog( - logDirectory: String, - closeFileAfterWrite: Boolean, - allowBatching: Boolean): Seq[String] = { - val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching) - val data = wal.readAll().asScala.map(byteBufferToString).toSeq - wal.close() - data - } - - /** Get the log files in a direction */ - def getLogFilesInDirectory(directory: String): Seq[String] = { - val logDirectoryPath = new Path(directory) - val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - - if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { - fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { - _.getName().split("-")(1).toLong - }.map { - _.toString.stripPrefix("file:") - } - } else { - Seq.empty - } - } - - def createWriteAheadLog( - logDirectory: String, - closeFileAfterWrite: Boolean, - allowBatching: Boolean): WriteAheadLog = { - val sparkConf = new SparkConf - val wal = new FileBasedWriteAheadLog(sparkConf, logDirectory, hadoopConf, 1, 1, - closeFileAfterWrite) - if (allowBatching) new BatchedWriteAheadLog(wal, sparkConf) else wal - } - - def generateRandomData(): Seq[String] = { - (1 to 100).map { _.toString } - } - - def readAndDeserializeDataManually(logFiles: Seq[String], allowBatching: Boolean): Seq[String] = { - if (allowBatching) { - logFiles.flatMap { file => - val data = readDataManually[Array[Array[Byte]]](file) - data.flatMap(byteArray => byteArray.map(Utils.deserialize[String])) - } - } else { - logFiles.flatMap { file => readDataManually[String](file)} - } - } - - implicit def stringToByteBuffer(str: String): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize(str)) - } - - implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { - Utils.deserialize[String](byteBuffer.array) - } } From 5f6f0206995626f83a5840acf0be5456137945f6 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 8 Nov 2015 21:15:02 -0800 Subject: [PATCH 22/26] fix scalastyle --- .../org/apache/spark/streaming/util/WriteAheadLogSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index fea124abe5436..eb6c5f472effd 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -407,7 +407,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val event4 = "is" val event5 = "doge" - writeEventWithFuture(batchedWal, event1, 3L) // 3 will automatically be flushed for the first write + // 3 will automatically be flushed for the first write + writeEventWithFuture(batchedWal, event1, 3L) // rest of the records will be batched while it takes 3 to get written writeEventWithFuture(batchedWal, event2, 5L) writeEventWithFuture(batchedWal, event3, 8L) From 6c422671ea986e814b8404b5e879002886bd6b6f Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 9 Nov 2015 09:16:49 -0800 Subject: [PATCH 23/26] add test and fix aggregation in test --- .../streaming/util/BatchedWriteAheadLog.scala | 6 +- .../streaming/util/WriteAheadLogSuite.scala | 66 +++++++++++++++---- 2 files changed, 58 insertions(+), 14 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 08bfb1517f6d5..9727ed2ba1445 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -172,6 +172,9 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp } buffer.foreach(_.promise.success(segment)) } catch { + case e: InterruptedException => + logWarning("BatchedWriteAheadLog Writer queue interrupted.", e) + buffer.foreach(_.promise.failure(e)) case NonFatal(e) => logWarning(s"BatchedWriteAheadLog Writer failed to write $buffer", e) buffer.foreach(_.promise.failure(e)) @@ -179,9 +182,6 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp buffer.clear() } } - - /** Method for querying the queue length during tests. */ - private[util] def getQueueLength(): Int = walWriteQueue.size() } /** Static methods for aggregating and de-aggregating records. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index eb6c5f472effd..f1a1881e7f334 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -79,7 +79,7 @@ abstract class CommonWriteAheadLogTests( val writtenData = (1 to 10).map { i => val data = generateRandomData() val file = testDir + s"/log-$i-$i" - writeDataManually(data, file) + writeDataManually(data, file, allowBatching) data }.flatten @@ -182,11 +182,14 @@ abstract class CommonWriteAheadLogTests( nonexistentTempPath.delete() assert(!nonexistentTempPath.exists()) - val writtenSegment = writeDataManually(generateRandomData(), testFile) - val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath, - new Configuration(), 1, 1, closeFileAfterWrite = false) + val writtenSegment = writeDataManually(generateRandomData(), testFile, allowBatching) + val wal = createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) assert(!nonexistentTempPath.exists(), "Directory created just by creating log object") - wal.read(writtenSegment.head) + if (allowBatching) { + intercept[UnsupportedOperationException](wal.read(writtenSegment.head)) + } else { + wal.read(writtenSegment.head) + } assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment") } } @@ -216,7 +219,7 @@ class FileBasedWriteAheadLogSuite test("FileBasedWriteAheadLogReader - sequentially reading data") { val writtenData = generateRandomData() - writeDataManually(writtenData, testFile) + writeDataManually(writtenData, testFile, allowBatching = false) val reader = new FileBasedWriteAheadLogReader(testFile, hadoopConf) val readData = reader.toSeq.map(byteBufferToString) assert(readData === writtenData) @@ -260,7 +263,7 @@ class FileBasedWriteAheadLogSuite test("FileBasedWriteAheadLogRandomReader - reading data using random reader") { // Write data manually for testing the random reader val writtenData = generateRandomData() - val segments = writeDataManually(writtenData, testFile) + val segments = writeDataManually(writtenData, testFile, allowBatching = false) // Get a random order of these segments and read them back val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10).flatten @@ -407,7 +410,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val event4 = "is" val event5 = "doge" - // 3 will automatically be flushed for the first write + // 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. writeEventWithFuture(batchedWal, event1, 3L) // rest of the records will be batched while it takes 3 to get written writeEventWithFuture(batchedWal, event2, 5L) @@ -437,6 +441,37 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( intercept[IllegalStateException](batchedWal.write(mock[ByteBuffer], 12L)) } + + 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 event1 = "hello" + val event2 = "world" + val event3 = "this" + val event4 = "is" + val event5 = "doge" + + val numFail = new AtomicInteger() + + // 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. + writeEventWithFuture(batchedWal, event1, 3L, null, numFail) + writeEventWithFuture(batchedWal, event2, 5L, null, numFail) + writeEventWithFuture(batchedWal, event3, 8L, null, numFail) + writeEventWithFuture(batchedWal, event4, 9L, null, numFail) + writeEventWithFuture(batchedWal, event5, 10L, null, numFail) + eventually(timeout(1 second)) { + assert(walBatchingThreadPool.getActiveCount === 5) + } + + batchedWal.close() + eventually(timeout(1 second)) { + assert(numFail.get() === 5) + } + } } class BatchedWriteAheadLogWithCloseFileAfterWriteSuite @@ -447,16 +482,25 @@ object WriteAheadLogSuite { private val hadoopConf = new Configuration() /** Write data to a file directly and return an array of the file segments written. */ - def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = { + def writeDataManually( + data: Seq[String], + file: String, + allowBatching: Boolean): Seq[FileBasedWriteAheadLogSegment] = { val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]() val writer = HdfsUtils.getOutputStream(file, hadoopConf) - data.foreach { item => + def writeToStream(bytes: Array[Byte]): Unit = { val offset = writer.getPos - val bytes = Utils.serialize(item) writer.writeInt(bytes.size) writer.write(bytes) segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size) } + if (allowBatching) { + writeToStream(wrapArrayArrayByte(data.toArray[String]).array()) + } else { + data.foreach { item => + writeToStream(Utils.serialize(item)) + } + } writer.close() segments } From 6461f4bfda022c2ce9850a162886c44fed75ee99 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 9 Nov 2015 10:30:06 -0800 Subject: [PATCH 24/26] remove synchronized --- .../apache/spark/streaming/scheduler/ReceiverTracker.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 207b3dc95a16e..0160113320f93 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -621,9 +621,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false override def onStop(): Unit = { submitJobThreadPool.shutdownNow() - synchronized { - active = false - } + active = false walBatchingThreadPool.shutdown() } From 7ec55c5f637a61a75e54456dfded5fe663c16a04 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 9 Nov 2015 13:34:50 -0800 Subject: [PATCH 25/26] address last comment --- .../streaming/scheduler/ReceiverTracker.scala | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 0160113320f93..ea5d12b50fcc5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -494,20 +494,15 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(successful) case AddBlock(receivedBlockInfo) => if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { - val f = Future { - if (active) { - addBlock(receivedBlockInfo) - } else { - throw new IllegalStateException("Receiver Tracker Endpoint shut down.") - } - }(walBatchingThreadPool) - f.onComplete { result => + walBatchingThreadPool.execute(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { if (active) { - context.reply(result.get) + context.reply(addBlock(receivedBlockInfo)) } else { - throw new IllegalStateException("Receiver Tracker Endpoint shut down.") + throw new IllegalStateException("ReceiverTracker RpcEndpoint shut down.") } - }(walBatchingThreadPool) + } + }) } else { context.reply(addBlock(receivedBlockInfo)) } From 1fe2e7bf6d428738944ec0880f848633e9c0b6c7 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 9 Nov 2015 14:56:57 -0800 Subject: [PATCH 26/26] update test --- .../streaming/util/WriteAheadLogSuite.scala | 56 ++++++++----------- 1 file changed, 24 insertions(+), 32 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index f1a1881e7f334..e96f4c2a29347 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.util import java.io._ import java.nio.ByteBuffer -import java.util.concurrent.ThreadPoolExecutor +import java.util.concurrent.{ExecutionException, ThreadPoolExecutor} import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConverters._ @@ -366,19 +366,13 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( } // we make the write requests in separate threads so that we don't block the test thread - private def writeEventWithFuture( - wal: WriteAheadLog, - event: String, - time: Long, - numSuccess: AtomicInteger = null, - numFail: AtomicInteger = null): Unit = { - val f = Future(wal.write(event, time))(walBatchingExecutionContext) - f.onComplete { - case Success(v) => - assert(v === walHandle) // return our mock handle after the write - if (numSuccess != null) numSuccess.incrementAndGet() - case Failure(v) => if (numFail != null) numFail.incrementAndGet() - }(walBatchingExecutionContext) + private def promiseWriteEvent(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = { + val p = Promise[Unit]() + p.completeWith(Future { + val v = wal.write(event, time) + assert(v === walHandle) + }(walBatchingExecutionContext)) + p } /** @@ -412,12 +406,12 @@ 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. - writeEventWithFuture(batchedWal, event1, 3L) + promiseWriteEvent(batchedWal, event1, 3L) // rest of the records will be batched while it takes 3 to get written - writeEventWithFuture(batchedWal, event2, 5L) - writeEventWithFuture(batchedWal, event3, 8L) - writeEventWithFuture(batchedWal, event4, 12L) - writeEventWithFuture(batchedWal, event5, 10L) + promiseWriteEvent(batchedWal, event2, 5L) + promiseWriteEvent(batchedWal, event3, 8L) + promiseWriteEvent(batchedWal, event4, 12L) + promiseWriteEvent(batchedWal, event5, 10L) eventually(timeout(1 second)) { assert(walBatchingThreadPool.getActiveCount === 5) } @@ -448,28 +442,26 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // block the write so that we can batch some records writeBlockingPromise(wal) - val event1 = "hello" - val event2 = "world" - val event3 = "this" - val event4 = "is" - val event5 = "doge" - - val numFail = new AtomicInteger() + val event1 = ("hello", 3L) + val event2 = ("world", 5L) + val event3 = ("this", 8L) + val event4 = ("is", 9L) + val event5 = ("doge", 10L) // 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. - writeEventWithFuture(batchedWal, event1, 3L, null, numFail) - writeEventWithFuture(batchedWal, event2, 5L, null, numFail) - writeEventWithFuture(batchedWal, event3, 8L, null, numFail) - writeEventWithFuture(batchedWal, event4, 9L, null, numFail) - writeEventWithFuture(batchedWal, event5, 10L, null, numFail) + val writePromises = Seq(event1, event2, event3, event4, event5).map { event => + promiseWriteEvent(batchedWal, event._1, event._2) + } + eventually(timeout(1 second)) { assert(walBatchingThreadPool.getActiveCount === 5) } batchedWal.close() eventually(timeout(1 second)) { - assert(numFail.get() === 5) + assert(writePromises.forall(_.isCompleted)) + assert(writePromises.forall(_.future.value.get.isFailure)) // all should have failed } } }