spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tdas <...@git.apache.org>
Subject [GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...
Date Thu, 29 Oct 2015 20:00:17 GMT
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43436559
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
---
    @@ -0,0 +1,191 @@
    +/*
    + * 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.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.
    + */
    +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(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))
    +      }
    +      buffer.clear()
    +    }
    +  }
    +}
    +
    +private[streaming] object BatchedWriteAheadLog {
    +  private[streaming] def aggregateRecords(records: Seq[RecordBuffer]): ByteBuffer = {
    +    ByteBuffer.wrap(Utils.serialize(
    +      CombinedReceivedBlockTrackerLogEvent(records.map(_.record.array()).toArray)))
    --- End diff --
    
    Why is this tied with `CombinedReceivedBlockTrackerLogEvent`? This whole thing is completely
inpdendent of the RBT.


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

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


Mime
View raw message