Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/9143#discussion_r43337200
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
---
@@ -235,8 +297,84 @@ 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 = {
--- End diff --
missing new line.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]