Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43810862
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
 ---
    @@ -0,0 +1,204 @@
    +/*
    + * 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, 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.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.
    + */
    +private[streaming] class BatchedWriteAheadLog(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
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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. The data may be 
aggregated, and the user
    +   * should de-aggregate using [[BatchedWriteAheadLog.deaggregate]]
    +   *
    +   * 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().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 = 
{
    +    parent.clean(threshTime, waitForCompletion)
    +  }
    +
    +
    +  /**
    +   * Stop the batched writer thread, fulfill promises with failures and 
close parent writer.
    +   */
    +  override def close(): Unit = {
    +    logInfo("BatchedWriteAheadLog shutting down.")
    +    active = false
    +    fulfillPromises()
    +    batchWriterThreadPool.shutdownNow()
    +    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)
    +    }
    +  }
    +
    +  /** Start the actual log writer on a separate thread. Visible 
(protected) for testing. */
    +  protected def startBatchedWriterThread(): Unit = {
    +    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.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +  }
    +
    +  /** 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.")
    +    }
    +    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(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.clear()
    +  }
    +}
    +
    +/** 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[Array[Array[Byte]]](records.map(_.record.array()).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.
    +   */
    +  private[streaming] def deaggregate(buffer: ByteBuffer): 
Array[ByteBuffer] = {
    --- End diff --
    
    The whole class is already `private[streaming]`


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to