Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -193,12 +193,15 @@ private[streaming] class ReceivedBlockTracker(
getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo
}

// Insert the recovered block-to-batch allocations and clear the queue of received blocks
// (when the blocks were originally allocated to the batch, the queue must have been cleared).
// Insert the recovered block-to-batch allocations and removes them from queue of
// received blocks.
def insertAllocatedBatch(batchTime: Time, allocatedBlocks: AllocatedBlocks) {
logTrace(s"Recovery: Inserting allocated batch for time $batchTime to " +
s"${allocatedBlocks.streamIdToAllocatedBlocks}")
streamIdToUnallocatedBlockQueues.values.foreach { _.clear() }
allocatedBlocks.streamIdToAllocatedBlocks.foreach {
case (streamId, allocatedBlocksInStream) =>
getReceivedBlockQueue(streamId).dequeueAll(allocatedBlocksInStream.toSet)
}
timeToAllocatedBlocks.put(batchTime, allocatedBlocks)
lastAllocatedBatchTime = batchTime
}
Expand Down Expand Up @@ -227,7 +230,7 @@ private[streaming] class ReceivedBlockTracker(
}

/** Write an update to the tracker to the write ahead log */
private def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = {
private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = {
if (isWriteAheadLogEnabled) {
logTrace(s"Writing record: $record")
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
import org.apache.spark.internal.Logging
import org.apache.spark.storage.StreamBlockId
import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult
import org.apache.spark.streaming.scheduler._
import org.apache.spark.streaming.scheduler.{AllocatedBlocks, _}
import org.apache.spark.streaming.util._
import org.apache.spark.streaming.util.WriteAheadLogSuite._
import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils}
Expand Down Expand Up @@ -94,6 +94,27 @@ class ReceivedBlockTrackerSuite
receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos
}

test("recovery with write ahead logs should remove only allocated blocks from received queue") {
val manualClock = new ManualClock
val batchTime = manualClock.getTimeMillis()

val tracker1 = createTracker(clock = manualClock)
tracker1.isWriteAheadLogEnabled should be (true)

val allocatedBlockInfos = generateBlockInfos()
val unallocatedBlockInfos = generateBlockInfos()
val receivedBlockInfos = allocatedBlockInfos ++ unallocatedBlockInfos
receivedBlockInfos.foreach { b => tracker1.writeToLog(BlockAdditionEvent(b)) }
val allocatedBlocks = AllocatedBlocks(Map(streamId -> allocatedBlockInfos))
tracker1.writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))
tracker1.stop()

val tracker2 = createTracker(clock = manualClock, recoverFromWriteAheadLog = true)
tracker2.getBlocksOfBatch(batchTime) shouldEqual allocatedBlocks.streamIdToAllocatedBlocks
tracker2.getUnallocatedBlocks(streamId) shouldEqual unallocatedBlockInfos
tracker2.stop()
}

test("recovery and cleanup with write ahead logs") {
val manualClock = new ManualClock
// Set the time increment level to twice the rotation interval so that every increment creates
Expand Down