Skip to content
Snippets Groups Projects
Commit 0ce6f9b2 authored by Burak Yavuz's avatar Burak Yavuz Committed by Tathagata Das
Browse files

[SPARK-11141][STREAMING] Batch ReceivedBlockTrackerLogEvents for WAL writes

When using S3 as a directory for WALs, the writes take too long. The driver gets very easily bottlenecked when multiple receivers send AddBlock events to the ReceiverTracker. This PR adds batching of events in the ReceivedBlockTracker so that receivers don't get blocked by the driver for too long.

cc zsxwing tdas

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9143 from brkyvz/batch-wal-writes.
parent 26062d22
No related branches found
No related tags found
No related merge requests found
......@@ -22,12 +22,13 @@ import java.nio.ByteBuffer
import scala.collection.JavaConverters._
import scala.collection.mutable
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.streaming.util.{BatchedWriteAheadLog, WriteAheadLog, WriteAheadLogUtils}
import org.apache.spark.util.{Clock, Utils}
import org.apache.spark.{Logging, SparkConf}
......@@ -41,7 +42,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]]) {
......@@ -82,15 +82,22 @@ 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 {
def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = {
try {
writeToLog(BlockAdditionEvent(receivedBlockInfo))
getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo
logDebug(s"Stream ${receivedBlockInfo.streamId} received " +
s"block ${receivedBlockInfo.blockStoreResult.blockId}")
true
val writeResult = writeToLog(BlockAdditionEvent(receivedBlockInfo))
if (writeResult) {
synchronized {
getReceivedBlockQueue(receivedBlockInfo.streamId) += 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.")
}
writeResult
} catch {
case e: Exception =>
case NonFatal(e) =>
logError(s"Error adding block $receivedBlockInfo", e)
false
}
......@@ -106,10 +113,12 @@ private[streaming] class ReceivedBlockTracker(
(streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true))
}.toMap
val allocatedBlocks = AllocatedBlocks(streamIdToBlocks)
writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))
timeToAllocatedBlocks(batchTime) = allocatedBlocks
lastAllocatedBatchTime = batchTime
allocatedBlocks
if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) {
timeToAllocatedBlocks.put(batchTime, allocatedBlocks)
lastAllocatedBatchTime = batchTime
} 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,
......@@ -157,9 +166,12 @@ private[streaming] class ReceivedBlockTracker(
require(cleanupThreshTime.milliseconds < clock.getTimeMillis())
val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq
logInfo("Deleting batches " + timesToCleanup)
writeToLog(BatchCleanupEvent(timesToCleanup))
timeToAllocatedBlocks --= timesToCleanup
writeAheadLogOption.foreach(_.clean(cleanupThreshTime.milliseconds, waitForCompletion))
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.")
}
}
/** Stop the block tracker. */
......@@ -185,8 +197,8 @@ 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)
lastAllocatedBatchTime = batchTime
}
// Cleanup the batch allocations
......@@ -213,12 +225,20 @@ private[streaming] class ReceivedBlockTracker(
}
/** 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())
logTrace(s"Writing record: $record")
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
}
}
......
......@@ -20,7 +20,7 @@ package org.apache.spark.streaming.scheduler
import java.util.concurrent.{CountDownLatch, TimeUnit}
import scala.collection.mutable.HashMap
import scala.concurrent.ExecutionContext
import scala.concurrent.{Future, ExecutionContext}
import scala.language.existentials
import scala.util.{Failure, Success}
......@@ -437,7 +437,12 @@ 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-thread-pool"))
@volatile private var active: Boolean = true
override def receive: PartialFunction[Any, Unit] = {
// Local messages
......@@ -488,7 +493,19 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
registerReceiver(streamId, typ, host, executorId, receiverEndpoint, context.senderAddress)
context.reply(successful)
case AddBlock(receivedBlockInfo) =>
context.reply(addBlock(receivedBlockInfo))
if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) {
walBatchingThreadPool.execute(new Runnable {
override def run(): Unit = Utils.tryLogNonFatalError {
if (active) {
context.reply(addBlock(receivedBlockInfo))
} else {
throw new IllegalStateException("ReceiverTracker RpcEndpoint shut down.")
}
}
})
} else {
context.reply(addBlock(receivedBlockInfo))
}
case DeregisterReceiver(streamId, message, error) =>
deregisterReceiver(streamId, message, error)
context.reply(true)
......@@ -599,6 +616,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
override def onStop(): Unit = {
submitJobThreadPool.shutdownNow()
active = false
walBatchingThreadPool.shutdown()
}
/**
......
/*
* 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
import java.util.concurrent.LinkedBlockingQueue
import java.util.{Iterator => JIterator}
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.{Await, Promise}
import scala.concurrent.duration._
import scala.util.control.NonFatal
import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.util.Utils
/**
* 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.
*
* 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.
*/
private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: SparkConf)
extends WriteAheadLog with Logging {
import BatchedWriteAheadLog._
private val walWriteQueue = new LinkedBlockingQueue[Record]()
// Whether the writer thread is active
@volatile private var active: Boolean = true
private val buffer = new ArrayBuffer[Record]()
private val batchedWriterThread = 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]()
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 IllegalStateException("close() was called on BatchedWriteAheadLog before " +
s"write request with time $time could be fulfilled.")
}
}
/**
* 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 = {
throw new UnsupportedOperationException("read() is not supported for BatchedWriteAheadLog " +
"as the data may require de-aggregation.")
}
/**
* 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
}
/**
* 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 = {
wrappedLog.clean(threshTime, waitForCompletion)
}
/**
* Stop the batched writer thread, fulfill promises with failures and close the wrapped WAL.
*/
override def close(): Unit = {
logInfo(s"BatchedWriteAheadLog shutting down at time: ${System.currentTimeMillis()}.")
synchronized {
active = false
}
batchedWriterThread.interrupt()
batchedWriterThread.join()
while (!walWriteQueue.isEmpty) {
val Record(_, time, promise) = walWriteQueue.poll()
promise.failure(new IllegalStateException("close() was called on BatchedWriteAheadLog " +
s"before write request with time $time could be fulfilled."))
}
wrappedLog.close()
}
/** 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) {
try {
flushRecords()
} catch {
case NonFatal(e) =>
logWarning("Encountered exception in Batched Writer Thread.", e)
}
}
logInfo("BatchedWriteAheadLog Writer thread exiting.")
}
}, "BatchedWriteAheadLog Writer")
thread.setDaemon(true)
thread.start()
thread
}
/** 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("BatchedWriteAheadLog 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 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))
} 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))
} finally {
buffer.clear()
}
}
}
/** Static methods for aggregating and de-aggregating records. */
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.
*/
case class Record(data: ByteBuffer, 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. */
def aggregate(records: Seq[Record]): ByteBuffer = {
ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](
records.map(record => getByteArray(record.data)).toArray))
}
/**
* 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.
*/
def deaggregate(buffer: ByteBuffer): Array[ByteBuffer] = {
try {
Utils.deserialize[Array[Array[Byte]]](getByteArray(buffer)).map(ByteBuffer.wrap)
} catch {
case _: ClassCastException => // users may restart a stream with batching enabled
Array(buffer)
}
}
}
......@@ -38,6 +38,8 @@ 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.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"
......@@ -64,6 +66,18 @@ private[streaming] object WriteAheadLogUtils extends Logging {
}
}
def isBatchingEnabled(conf: SparkConf, isDriver: Boolean): Boolean = {
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)
......@@ -115,7 +129,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)
......@@ -128,6 +142,11 @@ private[streaming] object WriteAheadLogUtils extends Logging {
getRollingIntervalSecs(sparkConf, isDriver), getMaxFailures(sparkConf, isDriver),
shouldCloseFileAfterWrite(sparkConf, isDriver))
}
if (isBatchingEnabled(sparkConf, isDriver)) {
new BatchedWriteAheadLog(wal, sparkConf)
} else {
wal
}
}
/** Instantiate the class, either using single arg constructor or zero arg constructor */
......
/*
* 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
import java.util
import scala.reflect.ClassTag
import org.apache.hadoop.conf.Configuration
import org.apache.spark.{SparkException, SparkConf, SparkFunSuite}
import org.apache.spark.util.Utils
class WriteAheadLogUtilsSuite extends SparkFunSuite {
import WriteAheadLogUtilsSuite._
private val logDir = Utils.createTempDir().getAbsolutePath()
private val hadoopConf = new Configuration()
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)
}
}
object WriteAheadLogUtilsSuite {
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()
}
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment