You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by brkyvz <gi...@git.apache.org> on 2015/10/16 02:33:55 UTC

[GitHub] spark pull request: [STREAMING] Batch ReceivedBlockTrackerLogEvent...

GitHub user brkyvz opened a pull request:

    https://github.com/apache/spark/pull/9143

    [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 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/brkyvz/spark batch-wal-writes

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/9143.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #9143
    
----
commit 8f92f10f4aa17b1a19a72e1c257273bb26080bb5
Author: Burak Yavuz <br...@gmail.com>
Date:   2015-10-13T20:40:37Z

    ready for testing

commit 78c6069477422d4984b7107db435245c811dbab9
Author: Burak Yavuz <br...@gmail.com>
Date:   2015-10-14T15:21:10Z

    save changes

commit ee36f8968354ecb14f1153b9bd7fb8f0d4bb9e1e
Author: Burak Yavuz <br...@gmail.com>
Date:   2015-10-16T00:21:56Z

    add more tests

----


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155239389
  
    **[Test build #45434 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45434/consoleFull)** for PR 9143 at commit [`1fe2e7b`](https://github.com/apache/spark/commit/1fe2e7bf6d428738944ec0880f848633e9c0b6c7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
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 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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154930734
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154353164
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45208/
    Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148561959
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43973255
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,135 +362,197 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */
    +class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite {
    +  import BatchedWriteAheadLog._
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +  test("serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +    val buffers = events.map(e => RecordBuffer(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    +    assert(deaggregate.toSeq === events)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") {
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    +  import BatchedWriteAheadLog._
     
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    +  // Class that will help us test batching.
    +  private class MockBatchedWriteAheadLog(
    +      parent: WriteAheadLog,
    +      writerThread: Thread = mmock[Thread]) extends BatchedWriteAheadLog(parent) {
     
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    +    override def startBatchedWriterThread(): Thread = writerThread
    +
    +    override def flushRecords(): Unit = {
    +      buffer.append(walWriteQueue.take())
    +      walWriteQueue.drainTo(buffer.asJava)
    +    }
    +
    +    def mockWrite(successful: Boolean): Seq[RecordBuffer] = {
    +      val records = buffer.toSeq
    +      buffer.foreach { case RecordBuffer(byteBuffer, time, promise) =>
    +        if (successful) promise.success(mmock[WriteAheadLogRecordHandle]) else promise.success(null)
           }
    +      buffer.clear()
    +      records
         }
    +
    +    def getQueueLength(): Int = walWriteQueue.size()
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
     
    +  private def waitUntilTrue(f: () => Int, value: Int): Boolean = {
    +    val timeOut = 2000
    +    val start = System.currentTimeMillis()
    +    var result = false
    +    while (!result && (System.currentTimeMillis() - start) < timeOut) {
    +      Thread.sleep(50)
    +      result = f() == value
    +    }
    +    result
    +  }
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +  import WriteAheadLogSuite._
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    +  test("BatchedWriteAheadLog - records get added to a queue") {
    +    val numSuccess = new AtomicInteger()
    +    val numFail = new AtomicInteger()
    +    val wal = new MockBatchedWriteAheadLog(mmock[FileBasedWriteAheadLog])
     
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    +    def getNumSuccess(): Int = numSuccess.get()
    +    def getNumFail(): Int = numFail.get()
    +
    +    val walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool"))
    +
    +    def eventFuture(event: String, time: Long): Unit = {
    +      val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +      f.onSuccess { case v =>
    +        if (v != null) numSuccess.incrementAndGet() else numFail.incrementAndGet()
    +      }(walBatchingThreadPool)
         }
     
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    +    assert(wal.getQueueLength === 0)
    --- End diff --
    
    Would be good to add inline comments to give some idea on whats being done.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43973937
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,135 +362,197 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */
    +class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite {
    +  import BatchedWriteAheadLog._
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +  test("serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +    val buffers = events.map(e => RecordBuffer(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    +    assert(deaggregate.toSeq === events)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") {
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    +  import BatchedWriteAheadLog._
     
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    +  // Class that will help us test batching.
    +  private class MockBatchedWriteAheadLog(
    +      parent: WriteAheadLog,
    +      writerThread: Thread = mmock[Thread]) extends BatchedWriteAheadLog(parent) {
     
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    +    override def startBatchedWriterThread(): Thread = writerThread
    +
    +    override def flushRecords(): Unit = {
    +      buffer.append(walWriteQueue.take())
    +      walWriteQueue.drainTo(buffer.asJava)
    +    }
    +
    +    def mockWrite(successful: Boolean): Seq[RecordBuffer] = {
    +      val records = buffer.toSeq
    +      buffer.foreach { case RecordBuffer(byteBuffer, time, promise) =>
    +        if (successful) promise.success(mmock[WriteAheadLogRecordHandle]) else promise.success(null)
           }
    +      buffer.clear()
    +      records
         }
    +
    +    def getQueueLength(): Int = walWriteQueue.size()
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
     
    +  private def waitUntilTrue(f: () => Int, value: Int): Boolean = {
    --- End diff --
    
    Can this be replaced with `eventually`??


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154962132
  
    Other than the test, things look LGTM. @zsxwing could you take another pass as well. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154931541
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44238297
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala ---
    @@ -0,0 +1,286 @@
    +/*
    + * 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.io.EOFException
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +import scala.language.{implicitConversions, postfixOps}
    +import scala.reflect.ClassTag
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.Path
    +import org.scalatest.PrivateMethodTester
    +
    +import org.apache.spark.{SparkException, SparkConf, SparkFunSuite}
    +import org.apache.spark.util.{ManualClock, Utils}
    +
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
    +  import WriteAheadLogSuite._
    +
    +  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 WriteAheadLogSuite {
    +
    +  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()
    +
    +  private val hadoopConf = new Configuration()
    +
    +  /** Write data to a file directly and return an array of the file segments written. */
    +  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    +    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    +    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    +    data.foreach { item =>
    +      val offset = writer.getPos
    +      val bytes = Utils.serialize(item)
    +      writer.writeInt(bytes.size)
    +      writer.write(bytes)
    +      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    +    }
    +    writer.close()
    +    segments
    +  }
    +
    +  /**
    +   * Write data to a file using the writer class and return an array of the file segments written.
    +   */
    +  def writeDataUsingWriter(
    +      filePath: String,
    +      data: Seq[String]): Seq[FileBasedWriteAheadLogSegment] = {
    +    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    +    val segments = data.map {
    +      item => writer.write(item)
    +    }
    +    writer.close()
    +    segments
    +  }
    +
    +  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    +  def writeDataUsingWriteAheadLog(
    +      logDirectory: String,
    +      data: Seq[String],
    +      closeFileAfterWrite: Boolean,
    +      allowBatching: Boolean,
    +      manualClock: ManualClock = new ManualClock,
    +      closeLog: Boolean = true,
    +      clockAdvanceTime: Int = 500): WriteAheadLog = {
    +    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    +    val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching)
    +
    +    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    +    data.foreach { item =>
    +      manualClock.advance(clockAdvanceTime)
    +      wal.write(item, manualClock.getTimeMillis())
    +    }
    +    if (closeLog) wal.close()
    +    wal
    +  }
    +
    +  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    +  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    +    segments.map { segment =>
    +      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    +      try {
    +        reader.seek(segment.offset)
    +        val bytes = new Array[Byte](segment.length)
    +        reader.readInt()
    +        reader.readFully(bytes)
    +        val data = Utils.deserialize[String](bytes)
    +        reader.close()
    +        data
    +      } finally {
    +        reader.close()
    +      }
    +    }
    +  }
    +
    +  /** Read all the data from a log file directly and return the list of byte buffers. */
    +  def readDataManually[T](file: String): Seq[T] = {
    +    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    +    val buffer = new ArrayBuffer[T]
    +    try {
    +      while (true) {
    +        // Read till EOF is thrown
    +        val length = reader.readInt()
    +        val bytes = new Array[Byte](length)
    +        reader.read(bytes)
    +        buffer += Utils.deserialize[T](bytes)
    +      }
    +    } catch {
    +      case ex: EOFException =>
    +    } finally {
    +      reader.close()
    +    }
    +    buffer
    +  }
    +
    +  /** Read all the data from a log file using reader class and return the list of byte buffers. */
    +  def readDataUsingReader(file: String): Seq[String] = {
    +    val reader = new FileBasedWriteAheadLogReader(file, hadoopConf)
    +    val readData = reader.toList.map(byteBufferToString)
    +    reader.close()
    +    readData
    +  }
    +
    +  /** Read all the data in the log file in a directory using the WriteAheadLog class. */
    +  def readDataUsingWriteAheadLog(
    +      logDirectory: String,
    +      closeFileAfterWrite: Boolean,
    +      allowBatching: Boolean): Seq[String] = {
    +    val wal = createWriteAheadLog(logDirectory, closeFileAfterWrite, allowBatching)
    +    val data = wal.readAll().asScala.map(byteBufferToString).toSeq
    +    wal.close()
    +    data
    +  }
    +
    +  /** Get the log files in a direction */
    --- End diff --
    
    nit: direction --> directory


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44187373
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    Right, if it is running. So from what I can understand the code in the `Future` itself runs in one thread, and the `onComplete` code runs in another. Are both these no longer safe to run or is it safe to add the block - my take is both are not safe to run.
    
    We should probably need synchronization to ensure this safety then 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152277470
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44619/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42721029
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +350,67 @@ 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 =>
    +      new Thread(runnable, "Batch WAL Writer").start()
    +      Runtime.getRuntime.addShutdownHook(new Thread {
    +        override def run(): Unit = runnable.stop()
    +      })
    +    }
    +    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: List[ReceivedBlockTrackerLogEvent]): Unit = {
    +      writeAheadLogOption.foreach { logManager =>
    +        if (records.nonEmpty) {
    +          logDebug(s"Batched ${records.length} records for WAL write")
    +          logManager.write(ByteBuffer.wrap(Utils.serialize(CombinedRBTLogEvent(records))), 
    +            clock.getTimeMillis())
    +        }
    +      }
    +    }
    +
    +    def stop(): Unit = {
    +      active = false
    +    }
    +
    +    private def flushRecords(): Unit = {
    +      val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]()
    +      while (!walWriteQueue.isEmpty) {
    +        buffer.append(walWriteQueue.poll())
    +      }
    +      val records = buffer.toList
    +      try {
    +        writeRecords(records)
    +        records.foreach(walWriteStatusMap.put(_, Success))
    +      } catch {
    +        case e: Exception =>
    +          logWarning(s"Batch WAL Writer failed to write $records")
    +          records.foreach(walWriteStatusMap.put(_, Fail))
    +      }
    +    }
    +
    +    override def run(): Unit = {
    +      while (active) {
    +        flushRecords()
    +        Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF)
    --- End diff --
    
    Of cause, if using `LinkedBlockingQueue`, you need to call `interrupt` in `stop` method and handle it correctly here.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43436836
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala ---
    @@ -268,3 +267,96 @@ class ReceivedBlockTrackerSuite
     
       implicit def timeToMillis(time: Time): Long = time.milliseconds
     }
    +
    +/**
    --- End diff --
    
    Remove this :)


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972449
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -19,59 +19,57 @@ package org.apache.spark.streaming.util
     import java.io._
     import java.nio.ByteBuffer
     import java.util
    +import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConverters._
     import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise, Future, ExecutionContext}
     import scala.concurrent.duration._
     import scala.language.{implicitConversions, postfixOps}
     import scala.reflect.ClassTag
     
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
    +import org.mockito.Matchers._
    +import org.mockito.Matchers.{eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.concurrent.Eventually
     import org.scalatest.concurrent.Eventually._
     import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar.{mock => mmock}
     
    -import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.streaming.scheduler._
    +import org.apache.spark.util.{ThreadUtils, ManualClock, Utils}
     import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
     
    -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
    -
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
       import WriteAheadLogSuite._
     
    -  val hadoopConf = new Configuration()
    -  var tempDir: File = null
    -  var testDir: String = null
    -  var testFile: String = null
    -  var writeAheadLog: FileBasedWriteAheadLog = null
    +  val logDir = Utils.createTempDir().getAbsolutePath()
    --- End diff --
    
    private


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43812681
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    > @zsxwing, with the reference to the context still be the same if done lazily?
    
    yes. It's safe to use `context` at any time before shutdown


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152031671
  
    **[Test build #44548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44548/consoleFull)** for PR 9143 at commit [`596cf8f`](https://github.com/apache/spark/commit/596cf8f841472aa07bfee5bca6508dfd9e4f14fc).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153913868
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152639107
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44702/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43446276
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -82,34 +85,45 @@ 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) {
    +        afterBlockAddAcknowledged(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
         }
       }
     
    +  /** Update in-memory state after block add event has been logged to WAL. */
    +  private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = synchronized {
    --- End diff --
    
    Does this one line thing need to be moved to a different function?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154341067
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44069967
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    --- End diff --
    
    +1 for `wrappedLog`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43435349
  
    --- 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"))
    --- End diff --
    
    Actually why do you even need a threadpool? You just need a thread.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148563601
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152025089
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152025063
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43451585
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -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}
    --- End diff --
    
    In this file, i dont see you using the BatchedWriteAheadLog any where? I dont see the change where you optionally create a BatchedWriteAheadLog instead of the usual. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44185944
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    Yes. It may be running when the threadpool is shutdown. The threadpool won't kill any running task.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44310193
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -599,6 +621,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
     
         override def onStop(): Unit = {
           submitJobThreadPool.shutdownNow()
    +      synchronized {
    --- End diff --
    
    `synchronized` is not necessary since `active` is `volatile`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155211465
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155240408
  
    test this please


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240234
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -58,49 +71,127 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         Utils.deleteRecursively(tempDir)
       }
     
    -  test("WriteAheadLogUtils - log selection and creation") {
    -    val logDir = Utils.createTempDir().getAbsolutePath()
    +  test(testPrefix + "read all logs") {
    +    // Write data manually for testing reading through WriteAheadLog
    +    val writtenData = (1 to 10).map { i =>
    +      val data = generateRandomData()
    +      val file = testDir + s"/log-$i-$i"
    +      writeDataManually(data, file)
    --- End diff --
    
    I am a little confused. This does not seem to generate files with batched data in them. How does this work when `allowBatching = true`? That is, how is `readDataUsingWriteAheadLog(... allowBatching = true)` working when the files dont have batched data?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43811640
  
    --- 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.
    --- End diff --
    
    nit: Also add the info that it returns null if it fails


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240691
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogUtilsSuite.scala ---
    @@ -0,0 +1,286 @@
    +/*
    + * 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.io.EOFException
    +import java.nio.ByteBuffer
    +import java.util
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +import scala.language.{implicitConversions, postfixOps}
    +import scala.reflect.ClassTag
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.Path
    +import org.scalatest.PrivateMethodTester
    +
    +import org.apache.spark.{SparkException, SparkConf, SparkFunSuite}
    +import org.apache.spark.util.{ManualClock, Utils}
    +
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
    +  import WriteAheadLogSuite._
    +
    +  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 WriteAheadLogSuite {
    --- End diff --
    
    Why is WriteAheadLogSuite in the file WriteAheadLogUtilsSuite. Should be in the file WriteAheadLogSuite. Only the stuff needed by the class WriteAheadLogUtilsSuite (like the MockWAL classes) should be in the WriteAheadLogSuite object.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43810744
  
    --- 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
    --- End diff --
    
    Better to make this @volatile so that we dont have worry about multi-threaded envs.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152637673
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44174435
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * 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.{SparkException, Logging}
    +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.
    + *
    + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog.
    + */
    +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  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]()
    +    walWriteQueue.offer(Record(byteBuffer, time, promise))
    +    Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
    --- End diff --
    
    I think it's better to add a configuration for `WAL_WRITE_STATUS_TIMEOUT`. The default value is 5 seconds. It may be too short considering now we write a batch of records.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240066
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    --- End diff --
    
    nit: This can be simply `mock[WriteAheadLog]` and mock[FileBasedWriteAheadLogSegment]`, isnt it?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43855085
  
    --- 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.
    --- End diff --
    
    I think that is a good idea rather than using nulls.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44239951
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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.{SparkConf, SparkException, Logging}
    +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.
    + *
    + * In addition, notice that the write method is still a blocking call. This will ensure that a
    + * receiver will not be able to submit multiple `AddBlock` calls, jeopardizing the ordering of data.
    + *
    + * 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 SparkException("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 SparkException("close() was called on BatchedWriteAheadLog before " +
    +        s"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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    --- End diff --
    
    It's already named "Batched WAL Writer"?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152030524
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43435919
  
    --- 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.
    --- End diff --
    
    No it is not, you do some stuff in this classes close as well.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151749322
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44502/
    Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43336941
  
    --- 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 {
    --- End diff --
    
    BatchLogWriter -> BatchedLogWriter


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43930999
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -42,7 +52,9 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
       var tempDir: File = null
       var testDir: String = null
       var testFile: String = null
    -  var writeAheadLog: FileBasedWriteAheadLog = null
    +  var writeAheadLog: WriteAheadLog = null
    --- End diff --
    
    Can you mark all of these fields as protected?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/9143


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154537041
  
    > @zsxwing The same receiver won't be able to send Block A and B, as the receiver will be blocked until it receives the reply for A. Different receivers may send A, and B. In that case, we may add B before A, and that should be fine.
    
    Got it. Thanks for clarification.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240881
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    --- End diff --
    
    I think this method can be available in a nice test utils trait. I can refactor those out later.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43970275
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    @harishreedharan Isnt adding a loop to put records in a new list just strictly more code and harder to read? 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155211188
  
    test this please


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44092102
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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
    --- End diff --
    
    @burak oops, sorry man, my bad!
    @brkyvz see above


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152639104
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155226674
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43970385
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    --- End diff --
    
    Good point. We should use List here.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42720841
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -195,31 +285,56 @@ private[streaming] class ReceivedBlockTracker(
           timeToAllocatedBlocks --= batchTimes
         }
     
    +    def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = {
    +      event match {
    +        case CombinedRBTLogEvent(events) => events.foreach(resolveEvent)
    +        case BlockAdditionEvent(receivedBlockInfo) =>
    +          insertAddedBlock(receivedBlockInfo)
    +        case BatchAllocationEvent(time, allocatedBlocks) =>
    +          insertAllocatedBatch(time, allocatedBlocks)
    +        case BatchCleanupEvent(batchTimes) =>
    +          cleanupBatches(batchTimes)
    +      }
    +    }
    +
         writeAheadLogOption.foreach { writeAheadLog =>
           logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}")
           writeAheadLog.readAll().asScala.foreach { byteBuffer =>
             logTrace("Recovering record " + byteBuffer)
    -        Utils.deserialize[ReceivedBlockTrackerLogEvent](
    -          byteBuffer.array, Thread.currentThread().getContextClassLoader) match {
    -          case BlockAdditionEvent(receivedBlockInfo) =>
    -            insertAddedBlock(receivedBlockInfo)
    -          case BatchAllocationEvent(time, allocatedBlocks) =>
    -            insertAllocatedBatch(time, allocatedBlocks)
    -          case BatchCleanupEvent(batchTimes) =>
    -            cleanupBatches(batchTimes)
    -        }
    +        resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent](
    +          byteBuffer.array, Thread.currentThread().getContextClassLoader))
           }
         }
       }
     
       /** 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 =>
    +        logTrace(s"Writing record: $record")
             logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis())
           }
         }
    +    true
    +  }
    +
    +  import WALWriteStatus._
    +
    +  /**
    +   * Adds LogEvents to a queue so that they can be batched and written to the WAL.
    +   * Exposed for testing.
    +   */
    +  private[streaming] def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = {
    +    if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled
    +    walWriteStatusMap.put(event, Pending)
    +    walWriteQueue.offer(event)
    +    var timedOut = false
    +    val start = clock.getTimeMillis()
    +    while (walWriteStatusMap.get(event) == Pending) {
    +      Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF)
    --- End diff --
    
    I'm not a fan of `Thread.sleep`. Could you change `walWriteStatusMap` to `new ConcurrentHashMap[ReceivedBlockTrackerLogEvent, Promise[Boolean]]()`? Then you can use `Promise` to notify the write status. And this method can be changed to:
    ```
      private[streaming] def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = {
        if (!isWriteAheadLogEnabled) return true // return early if WAL is not enabled
        val promise = Promise[Boolean]()
        walWriteStatusMap.put(event, promise)
        walWriteQueue.offer(event)
        Await.result(promise.future.recover { case _ => false }, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
      }
    ```
    And you can also remove `WALWriteStatus` class.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148561455
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
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 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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152274143
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240131
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    -  }
    -
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    -    }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
           }
    -    }
    +    )
    +    promise
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    +    // rest of the records will be batched while it takes 3 to get written
    +    eventFuture(wal, event2, 5L)
    +    eventFuture(wal, event3, 8L)
    +    eventFuture(wal, event4, 12L)
    +    eventFuture(wal, event5, 10L)
    +    promise.success(true)
    +
    +    eventually(Eventually.timeout(1 second)) {
    +      verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L))
    +      // the file name should be the timestamp of the last record, as events should be naturally
    +      // in order of timestamp, and we need the last element.
    +      verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(10L))
         }
    -    buffer
    -  }
    -
    -  /** Read all the data from a log file using reader class and return the list of byte buffers. */
    -  def readDataUsingReader(file: String): Seq[String] = {
    -    val reader = new FileBasedWriteAheadLogReader(file, hadoopConf)
    -    val readData = reader.toList.map(byteBufferToString)
    -    reader.close()
    -    readData
       }
     
    -  /** Read all the data in the log file in a directory using the WriteAheadLog class. */
    -  def readDataUsingWriteAheadLog(logDirectory: String): Seq[String] = {
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite = false)
    -    val data = wal.readAll().asScala.map(byteBufferToString).toSeq
    +  test("BatchedWriteAheadLog - shutdown properly") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
         wal.close()
    -    data
    -  }
    -
    -  /** Get the log files in a direction */
    -  def getLogFilesInDirectory(directory: String): Seq[String] = {
    -    val logDirectoryPath = new Path(directory)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -
    -    if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) {
    -      fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy {
    -        _.getName().split("-")(1).toLong
    -      }.map {
    -        _.toString.stripPrefix("file:")
    -      }
    -    } else {
    -      Seq.empty
    -    }
    -  }
    -
    -  def generateRandomData(): Seq[String] = {
    -    (1 to 100).map { _.toString }
    -  }
    -
    -  implicit def stringToByteBuffer(str: String): ByteBuffer = {
    -    ByteBuffer.wrap(Utils.serialize(str))
    -  }
    -
    -  implicit def byteBufferToString(byteBuffer: ByteBuffer): String = {
    -    Utils.deserialize[String](byteBuffer.array)
    +    verify(fileBasedWAL, times(1)).close()
    --- End diff --
    
    I think this unit test should also test that if the batchWAL is closed, then outstanding write requests will complete with error, and not get blocked indefinitely. Very crucial behavior to avoid hard-to-debug deadlocks.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154342398
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972512
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -19,59 +19,57 @@ package org.apache.spark.streaming.util
     import java.io._
     import java.nio.ByteBuffer
     import java.util
    +import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConverters._
     import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise, Future, ExecutionContext}
     import scala.concurrent.duration._
     import scala.language.{implicitConversions, postfixOps}
     import scala.reflect.ClassTag
     
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
    +import org.mockito.Matchers._
    +import org.mockito.Matchers.{eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.concurrent.Eventually
     import org.scalatest.concurrent.Eventually._
     import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar.{mock => mmock}
     
    -import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.streaming.scheduler._
    +import org.apache.spark.util.{ThreadUtils, ManualClock, Utils}
     import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
     
    -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
    -
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
       import WriteAheadLogSuite._
     
    -  val hadoopConf = new Configuration()
    -  var tempDir: File = null
    -  var testDir: String = null
    -  var testFile: String = null
    -  var writeAheadLog: FileBasedWriteAheadLog = null
    +  val logDir = Utils.createTempDir().getAbsolutePath()
     
    -  before {
    -    tempDir = Utils.createTempDir()
    -    testDir = tempDir.toString
    -    testFile = new File(tempDir, "testFile").toString
    -    if (writeAheadLog != null) {
    -      writeAheadLog.close()
    -      writeAheadLog = null
    +  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].parent
    +      assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass)
    +    } else {
    +      assert(log.getClass === implicitly[ClassTag[T]].runtimeClass)
         }
    +    log
       }
     
    -  after {
    -    Utils.deleteRecursively(tempDir)
    +  def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = {
    +    val log = WriteAheadLogUtils.createLogForReceiver(conf, logDir, hadoopConf)
    +    assert(log.getClass === implicitly[ClassTag[T]].runtimeClass)
    +    log
       }
     
       test("WriteAheadLogUtils - log selection and creation") {
    --- End diff --
    
    The tag `WriteAheadLogUtils` is not needed any more since we have a separate test suite.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148561943
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44173447
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    Is it safe to just catch NonFatal exception and log it in Future body and `onComplete`?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151749184
  
    **[Test build #44502 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44502/consoleFull)** for PR 9143 at commit [`80a0b8d`](https://github.com/apache/spark/commit/80a0b8d9e994ccf5c9381e12dae4c736ad6c3800).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43446421
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -82,34 +85,45 @@ 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) {
    +        afterBlockAddAcknowledged(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
         }
       }
     
    +  /** Update in-memory state after block add event has been logged to WAL. */
    +  private def afterBlockAddAcknowledged(receivedBlockInfo: ReceivedBlockInfo): Unit = synchronized {
    +    getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo
    +  }
    +
       /**
        * Allocate all unallocated blocks to the given batch.
        * This event will get written to the write ahead log (if enabled).
        */
    -  def allocateBlocksToBatch(batchTime: Time): Unit = synchronized {
    +  def allocateBlocksToBatch(batchTime: Time): Unit = {
         if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) {
           val streamIdToBlocks = streamIds.map { streamId =>
               (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true))
    --- End diff --
    
    This dequeing needs to be done in a synchronized manner.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154353160
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43963503
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    This is really unnecessary. We should really not have to convert to a scala iterator, just to do a flatMap and then back to a Java one. Just having a loop adding all of the records map is not that horrible. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153911927
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155204883
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154533740
  
    @zsxwing The same receiver won't be able to send Block A and B, as the receiver will be blocked until it receives the reply. Different receivers may send A, and B. In that case, we may add B before A, and that should be fine.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155226564
  
    **[Test build #45423 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45423/consoleFull)** for PR 9143 at commit [`7ec55c5`](https://github.com/apache/spark/commit/7ec55c5f637a61a75e54456dfded5fe663c16a04).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155148740
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240091
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    -  }
    -
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    -    }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
           }
    -    }
    +    )
    +    promise
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    --- End diff --
    
    The `queue.take()` immediately takes the 3, and there is nothing left in the queue at that moment. Then the promise blocks the writing of `3`. The rest get queued.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43445514
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -102,6 +105,114 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
       }
     
    +  test("WriteAheadLog - read rotating logs") {
    --- End diff --
    
    Wait, do you even need it if different suites already show up different names. So probably we dont need "WriteAheadLog - " prefix at all. See if it makes sense.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43436097
  
    --- 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()
    --- End diff --
    
    You will have to mark all the pending promises in the queue as failed, so that thread blocked on them get unblocked with failures.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972351
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala ---
    @@ -180,8 +180,7 @@ class ReceivedBlockTrackerSuite
         // Cleanup first batch but not second batch
         val oldestLogFile = getWriteAheadLogFiles().head
         incrementTime()
    -    tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true)
    --- End diff --
    
    why was this line changed


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153440723
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43811779
  
    --- 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))
    --- End diff --
    
    The content of the ByteBuffer may be just a part of the array, e.g., this is a sliced ByteBuffer. It's better to copy the content to a new array, such as 
    ```
    val byteArray = new Array[Byte](buffer.record.remaining())
    buffer.record.get(byteArray)
    byteArray
    ```


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152304942
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154537412
  
    @zsxwing Maybe I should add that to the Javadoc.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155255206
  
    Thank you very much for all the feedback @tdas @zsxwing and @harishreedharan . This is definitely my most commented on PR. Even the most commented on per line even :S


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43337219
  
    --- 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 = {
    +        val promise = walWriteStatusMap.get(record)
    +        if (promise == null) {
    +          logError(s"Promise for writing record $record not found in status map!")
    +        } else {
    +          promise.success(successful)
    +        }
    +      }
    +      try {
    --- End diff --
    
    nit: add newline.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44239938
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    -  }
    -
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    -    }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
           }
    -    }
    +    )
    +    promise
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    +    // rest of the records will be batched while it takes 3 to get written
    +    eventFuture(wal, event2, 5L)
    +    eventFuture(wal, event3, 8L)
    +    eventFuture(wal, event4, 12L)
    +    eventFuture(wal, event5, 10L)
    +    promise.success(true)
    +
    +    eventually(Eventually.timeout(1 second)) {
    +      verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L))
    +      // the file name should be the timestamp of the last record, as events should be naturally
    +      // in order of timestamp, and we need the last element.
    +      verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(10L))
         }
    --- End diff --
    
    Shouldnt you verify that the futures have been completed with some expected WAL segment?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155132675
  
    **[Test build #45381 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45381/consoleFull)** for PR 9143 at commit [`6c42267`](https://github.com/apache/spark/commit/6c422671ea986e814b8404b5e879002886bd6b6f).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154262678
  
    **[Test build #45183 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45183/consoleFull)** for PR 9143 at commit [`ec6d8da`](https://github.com/apache/spark/commit/ec6d8da236c6a742ae523a81ebe4754013dc4a59).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43947286
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -599,6 +607,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
     
         override def onStop(): Unit = {
           submitJobThreadPool.shutdownNow()
    +      walBatchingThreadPool.shutdownNow()
    --- End diff --
    
    `shutdown` makes more sense here actually. Thanks


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154546987
  
    **[Test build #45257 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45257/consoleFull)** for PR 9143 at commit [`21f44ea`](https://github.com/apache/spark/commit/21f44eaf793cfbbbafc7573583c645feab44af3f).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155241173
  
    **[Test build #45445 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45445/consoleFull)** for PR 9143 at commit [`1fe2e7b`](https://github.com/apache/spark/commit/1fe2e7bf6d428738944ec0880f848633e9c0b6c7).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152277468
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154544394
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154351047
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45205/
    Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43970391
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -439,6 +439,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
         private val submitJobThreadPool = ExecutionContext.fromExecutorService(
           ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool"))
     
    +    private val walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool"))
    --- End diff --
    
    good catch.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43963915
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    > Just having a loop adding all of the records into a new List is not that horrible.
    
    This reads lazily and can reduce the peak memory.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972618
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -19,59 +19,57 @@ package org.apache.spark.streaming.util
     import java.io._
     import java.nio.ByteBuffer
     import java.util
    +import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConverters._
     import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise, Future, ExecutionContext}
     import scala.concurrent.duration._
     import scala.language.{implicitConversions, postfixOps}
     import scala.reflect.ClassTag
     
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
    +import org.mockito.Matchers._
    --- End diff --
    
    import ordering is messed up.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152289433
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155254346
  
    **[Test build #45445 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45445/consoleFull)** for PR 9143 at commit [`1fe2e7b`](https://github.com/apache/spark/commit/1fe2e7bf6d428738944ec0880f848633e9c0b6c7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154930295
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43964535
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    I really don't think we are dealing with all that much memory. I feel we are over-using the conversions here with very little additional benefit. All we are doing is a flatMap. 
    
    Moreover, the flatMap is going to allocate a new collection/seq - so a loop is probably going to use less memory, because you are adding only the newly generated `ByteBuffers`. There are no new objects allocated other than the ones required (in the last comment when I said records, I meant the Buffers that were created as a result of the deaggregation, not all of the data from the `readAll`)


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43964698
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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
    --- End diff --
    
    Ah, ok. Thanks!


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44243570
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -58,49 +71,127 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         Utils.deleteRecursively(tempDir)
       }
     
    -  test("WriteAheadLogUtils - log selection and creation") {
    -    val logDir = Utils.createTempDir().getAbsolutePath()
    +  test(testPrefix + "read all logs") {
    +    // Write data manually for testing reading through WriteAheadLog
    +    val writtenData = (1 to 10).map { i =>
    +      val data = generateRandomData()
    +      val file = testDir + s"/log-$i-$i"
    +      writeDataManually(data, file)
    --- End diff --
    
    Actually, we dont guarantee cross release compatibility. So achieving that is less important. Its more important to actually test the base case with batching, write batched data and read batched data. Similar to the FileBasedWALSuite where this unit test tests the reader's ability to read manually/directly written data, i was expecting this same unit test for BatchedWALSuite to test reading of batched data by reading directly generated batched data.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152027123
  
    **[Test build #44546 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44546/consoleFull)** for PR 9143 at commit [`48c9f22`](https://github.com/apache/spark/commit/48c9f2244d0ab21f53ab28b6cd77ed8238d397d5).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42719627
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +350,67 @@ 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 =>
    +      new Thread(runnable, "Batch WAL Writer").start()
    +      Runtime.getRuntime.addShutdownHook(new Thread {
    +        override def run(): Unit = runnable.stop()
    +      })
    +    }
    +    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: List[ReceivedBlockTrackerLogEvent]): Unit = {
    +      writeAheadLogOption.foreach { logManager =>
    +        if (records.nonEmpty) {
    +          logDebug(s"Batched ${records.length} records for WAL write")
    +          logManager.write(ByteBuffer.wrap(Utils.serialize(CombinedRBTLogEvent(records))), 
    +            clock.getTimeMillis())
    +        }
    +      }
    +    }
    +
    +    def stop(): Unit = {
    +      active = false
    +    }
    +
    +    private def flushRecords(): Unit = {
    +      val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]()
    +      while (!walWriteQueue.isEmpty) {
    +        buffer.append(walWriteQueue.poll())
    +      }
    +      val records = buffer.toList
    +      try {
    +        writeRecords(records)
    +        records.foreach(walWriteStatusMap.put(_, Success))
    +      } catch {
    +        case e: Exception =>
    --- End diff --
    
    nit: Exception -> NonFatal(e)


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155254492
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43438534
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala ---
    @@ -268,3 +267,96 @@ class ReceivedBlockTrackerSuite
     
       implicit def timeToMillis(time: Time): Long = time.milliseconds
     }
    +
    +/**
    --- End diff --
    
    I will. Still WIP


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155240395
  
    Failed to a flaky test. Restarting


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154871950
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154341042
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154258385
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43337370
  
    --- 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 = {
    --- End diff --
    
    add scala docs, saying what this does. In fact, its not obvious from the name, what is the functional difference between flushRecords, and writeRecords?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155228476
  
    **[Test build #45434 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45434/consoleFull)** for PR 9143 at commit [`1fe2e7b`](https://github.com/apache/spark/commit/1fe2e7bf6d428738944ec0880f848633e9c0b6c7).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155172748
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972721
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -19,59 +19,57 @@ package org.apache.spark.streaming.util
     import java.io._
     import java.nio.ByteBuffer
     import java.util
    +import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConverters._
     import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise, Future, ExecutionContext}
     import scala.concurrent.duration._
     import scala.language.{implicitConversions, postfixOps}
     import scala.reflect.ClassTag
     
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
    +import org.mockito.Matchers._
    +import org.mockito.Matchers.{eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.concurrent.Eventually
     import org.scalatest.concurrent.Eventually._
     import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar.{mock => mmock}
     
    -import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.streaming.scheduler._
    +import org.apache.spark.util.{ThreadUtils, ManualClock, Utils}
     import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
     
    -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
    -
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
       import WriteAheadLogSuite._
     
    -  val hadoopConf = new Configuration()
    -  var tempDir: File = null
    -  var testDir: String = null
    -  var testFile: String = null
    -  var writeAheadLog: FileBasedWriteAheadLog = null
    +  val logDir = Utils.createTempDir().getAbsolutePath()
     
    -  before {
    -    tempDir = Utils.createTempDir()
    -    testDir = tempDir.toString
    -    testFile = new File(tempDir, "testFile").toString
    -    if (writeAheadLog != null) {
    -      writeAheadLog.close()
    -      writeAheadLog = null
    +  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].parent
    +      assert(parentLog.getClass === implicitly[ClassTag[T]].runtimeClass)
    +    } else {
    +      assert(log.getClass === implicitly[ClassTag[T]].runtimeClass)
         }
    +    log
       }
     
    -  after {
    -    Utils.deleteRecursively(tempDir)
    +  def assertReceiverLogClass[T: ClassTag](conf: SparkConf): WriteAheadLog = {
    --- End diff --
    
    This was a bug from earlier, but could you update `T: ClassTag` to match the one in `assertDriverLogClass`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44182750
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,187 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    -
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    -
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
    +      }
    +    )
    +    promise
       }
     
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    +  test("BatchedWriteAheadLog - records get added to a queue") {
    +    val numSuccess = new AtomicInteger()
    +    val numFail = new AtomicInteger()
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL)
    +
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    // make sure queue is empty initially
    +    assert(wal.getQueueLength === 0)
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 5L, numSuccess, numFail)
    +    eventFuture(wal, event2, 10L, numSuccess, numFail)
    +    eventFuture(wal, event3, 11L, numSuccess, numFail)
    +    eventFuture(wal, event4, 12L, numSuccess, numFail)
    +    eventFuture(wal, event5, 20L, numSuccess, numFail)
    +
    +    eventually(Eventually.timeout(2 seconds)) {
    +      // the first element will immediately be taken and the rest will get queued
    +      assert(wal.getQueueLength() == 4)
         }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    -      }
    +    assert(numSuccess.get() === 0)
    +    assert(numFail.get() === 0)
    +    // remove block so that the writes are made
    +    promise.success(null)
    +
    +    eventually(Eventually.timeout(2 seconds)) {
    +      assert(wal.getQueueLength() == 0)
    +      assert(numSuccess.get() === 5)
    +      assert(numFail.get() == 0)
         }
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    +    // rest of the records will be batched while it takes 3 to get written
    +    eventFuture(wal, event2, 5L)
    +    eventFuture(wal, event3, 8L)
    +    eventFuture(wal, event4, 12L)
    +    eventFuture(wal, event5, 10L)
    +    promise.success(true)
    +
    +    verify(fileBasedWAL, times(1)).write(any[ByteBuffer], meq(3L))
    --- End diff --
    
    This line should also be in an `eventually` block in case Jenkins is extremely slow.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154351044
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155129633
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44179183
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * 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.{SparkException, Logging}
    +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.
    + *
    + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog.
    + */
    +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  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]()
    +    walWriteQueue.offer(Record(byteBuffer, time, promise))
    +    Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
    +  }
    +
    +  /**
    +   * 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 = {
    +    wrappedLog.read(segment)
    --- End diff --
    
    Maybe throw an exception here? E.g., `throw new UnsupportedOperationException("BatchedWriteAheadLog.read")`. To make the error more explicit if BatchedWriteAheadLog is used in Executor by mistake.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43336812
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -73,6 +79,18 @@ private[streaming] class ReceivedBlockTracker(
       private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue]
       private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks]
       private val writeAheadLogOption = createWriteAheadLog()
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[ReceivedBlockTrackerLogEvent]()
    --- End diff --
    
    Rather than have the corresponding promise in another map, why not just keep the event+promise in the queue? Wouldnt it be a simpler design?



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44084903
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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])
    --- End diff --
    
    This promise looks super weird in a class called `RecordBuffer`. This class could be called `RecordTracker` or something.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42721825
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -195,31 +285,56 @@ private[streaming] class ReceivedBlockTracker(
           timeToAllocatedBlocks --= batchTimes
         }
     
    +    def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = {
    +      event match {
    +        case CombinedRBTLogEvent(events) => events.foreach(resolveEvent)
    +        case BlockAdditionEvent(receivedBlockInfo) =>
    +          insertAddedBlock(receivedBlockInfo)
    +        case BatchAllocationEvent(time, allocatedBlocks) =>
    +          insertAllocatedBatch(time, allocatedBlocks)
    +        case BatchCleanupEvent(batchTimes) =>
    +          cleanupBatches(batchTimes)
    +      }
    +    }
    +
         writeAheadLogOption.foreach { writeAheadLog =>
           logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}")
           writeAheadLog.readAll().asScala.foreach { byteBuffer =>
             logTrace("Recovering record " + byteBuffer)
    -        Utils.deserialize[ReceivedBlockTrackerLogEvent](
    -          byteBuffer.array, Thread.currentThread().getContextClassLoader) match {
    -          case BlockAdditionEvent(receivedBlockInfo) =>
    -            insertAddedBlock(receivedBlockInfo)
    -          case BatchAllocationEvent(time, allocatedBlocks) =>
    -            insertAllocatedBatch(time, allocatedBlocks)
    -          case BatchCleanupEvent(batchTimes) =>
    -            cleanupBatches(batchTimes)
    -        }
    +        resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent](
    +          byteBuffer.array, Thread.currentThread().getContextClassLoader))
           }
         }
       }
     
       /** 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 =>
    +        logTrace(s"Writing record: $record")
             logManager.write(ByteBuffer.wrap(Utils.serialize(record)), clock.getTimeMillis())
           }
         }
    +    true
    +  }
    +
    +  import WALWriteStatus._
    +
    +  /**
    +   * Adds LogEvents to a queue so that they can be batched and written to the WAL.
    +   * Exposed for testing.
    +   */
    +  private[streaming] def writeToLogAsync(event: ReceivedBlockTrackerLogEvent): Boolean = {
    --- End diff --
    
    `writeToLogAsync` looks confusing. Actually, `async` usually means putting the work in another thread and return immediately, and it should run pretty fast. But in this method, it actually needs to wait for the result.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152035556
  
    **[Test build #44550 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44550/consoleFull)** for PR 9143 at commit [`13cab44`](https://github.com/apache/spark/commit/13cab44bcd1ed4f5c1fa5bb855375f137fa15768).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42719713
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -40,7 +42,8 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks:
       extends ReceivedBlockTrackerLogEvent
     private[streaming] case class BatchCleanupEvent(times: Seq[Time])
       extends ReceivedBlockTrackerLogEvent
    -
    +private[streaming] case class CombinedRBTLogEvent(events: List[ReceivedBlockTrackerLogEvent])
    --- End diff --
    
    Could you use `Seq` rather than `List`? Then you don't need to convert an `ArrayBuffer` to `List`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43452869
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -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}
    --- End diff --
    
    NVM, i found it. WALUtils doing the wrapping is perfect.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43438667
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -102,6 +105,114 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
       }
     
    +  test("WriteAheadLog - read rotating logs") {
    --- End diff --
    
    Can you make this "WriteAheadLog" parameterized?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155239425
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154930525
  
    **[Test build #45341 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45341/consoleFull)** for PR 9143 at commit [`c659cd7`](https://github.com/apache/spark/commit/c659cd7cfd9ef011f16583752aea4fcd244c3e02).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44234591
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -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"))
    +
    +    private var active: Boolean = true
    --- End diff --
    
    Needs `@volatile` !


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240746
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    --- End diff --
    
    eventFuture --> writeEventWithFuture


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152304761
  
    **[Test build #44627 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44627/consoleFull)** for PR 9143 at commit [`e288cb2`](https://github.com/apache/spark/commit/e288cb2382d32ced61e7ae6f81a3b7cca2815eae).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155172521
  
    **[Test build #45387 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45387/consoleFull)** for PR 9143 at commit [`6461f4b`](https://github.com/apache/spark/commit/6461f4bfda022c2ce9850a162886c44fed75ee99).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152026598
  
    **[Test build #44546 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44546/consoleFull)** for PR 9143 at commit [`48c9f22`](https://github.com/apache/spark/commit/48c9f2244d0ab21f53ab28b6cd77ed8238d397d5).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153440724
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44937/
    Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44193051
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    @tdas what do you think about the race condition?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148563173
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43823/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152027128
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44546/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43337296
  
    --- 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 = {
    +        val promise = walWriteStatusMap.get(record)
    +        if (promise == null) {
    +          logError(s"Promise for writing record $record not found in status map!")
    +        } else {
    +          promise.success(successful)
    +        }
    +      }
    +      try {
    +        writeRecords(buffer)
    +        buffer.foreach(updateRecordStatus(_, successful = true))
    +      } catch {
    +        case NonFatal(e) =>
    +          logWarning(s"Batch WAL Writer failed to write $buffer", e)
    +          buffer.foreach(updateRecordStatus(_, successful = false))
    +      }
    +    }
    +
    +    override def run(): Unit = {
    --- End diff --
    
    nit: would be nice to order the public methods above the private methods. easier to read the control flow.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44174052
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * 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.{SparkException, Logging}
    +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.
    + *
    + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog.
    + */
    +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    --- End diff --
    
    nit: not necessary


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44087292
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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 = {
    --- End diff --
    
    Wasnt there a comment earlier saying that we should not be returning null for failed writes, we should throw exceptions, and let the higher layer handle failures according to their semantics.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43451078
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -195,30 +219,35 @@ private[streaming] class ReceivedBlockTracker(
           timeToAllocatedBlocks --= batchTimes
         }
     
    +    def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = {
    --- End diff --
    
    This does not need to change. The RBT should not even be aware that there is batching going on in the WAL that is being used. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154262703
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152277458
  
    **[Test build #44619 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44619/consoleFull)** for PR 9143 at commit [`ea62d25`](https://github.com/apache/spark/commit/ea62d253f7e8d33484ab19263a7b61968b774d67).
     * This patch **fails RAT tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153416242
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148561614
  
      [Test build #43821 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43821/consoleFull) for   PR 9143 at commit [`ee36f89`](https://github.com/apache/spark/commit/ee36f8968354ecb14f1153b9bd7fb8f0d4bb9e1e).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154956060
  
    **[Test build #45345 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45345/consoleFull)** for PR 9143 at commit [`5f6f020`](https://github.com/apache/spark/commit/5f6f0206995626f83a5840acf0be5456137945f6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43436429
  
    --- 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(
    --- End diff --
    
    I think this class should be internal to BatchedWAL, as this is used only by internal implementations of the BatchedWAL.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152274223
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155246265
  
    > Test build #45434 has finished for PR 9143 at commit 1fe2e7b.
    > 
    > This patch fails Spark unit tests.
    > This patch merges cleanly.
    > This patch adds the following public classes (experimental):\n * case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])\n
    
    Should we move `RawTextSender/RateLimitedOutputStream` to `examples`? It doesn't make sense to keep them in `streaming` folder.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43971355
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -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))
    --- End diff --
    
    Why will it never get cleaned up? If any of the future attempts with higher cleanupThreshTime than the current one, will clean up older files. 
    
    BTW, even if you call `WAL.clean` before `timeToAllocatedBlocks --= timesToCleanup`, if `waitForCompletion = false`  (which is the case in production), then the files are going to be deleted asynchronously and there are not guarantees anyway. 
    
    So I dont see much of a difference in moving them around. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152031952
  
    **[Test build #44548 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44548/consoleFull)** for PR 9143 at commit [`596cf8f`](https://github.com/apache/spark/commit/596cf8f841472aa07bfee5bca6508dfd9e4f14fc).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44084845
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    parent.close()
    +  }
    +
    +  /**
    +   * Respond to any promises that may have been left in the queue, to unblock receivers during
    +   * shutdown.
    +   */
    +  private def fulfillPromises(): Unit = {
    --- End diff --
    
    Well, BatchWAL is supposed to be unaware of receivers, so cant name that. Why even make a separate function, its small, called from just one place. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43436334
  
    --- 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)))
    +  }
    +
    +  private[streaming] def deaggregate(
    --- End diff --
    
    inconsistent with `aggregateRecords`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155211487
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43973837
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,135 +362,197 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */
    +class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite {
    --- End diff --
    
    Why is this a separate suite?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152034528
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43813114
  
    --- 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),
    --- End diff --
    
    You can use `ThreadUtils.sameThread` here. It's safe to run `case _ => null` in the `BatchedWriterThread`. Then `batchWriterThreadPool` can be removed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155129611
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148561443
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154350950
  
    **[Test build #45205 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45205/consoleFull)** for PR 9143 at commit [`f554698`](https://github.com/apache/spark/commit/f55469857ef4dc94ddf83ce29c89f473f5ba0c5b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42720937
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +350,67 @@ 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 =>
    +      new Thread(runnable, "Batch WAL Writer").start()
    +      Runtime.getRuntime.addShutdownHook(new Thread {
    +        override def run(): Unit = runnable.stop()
    +      })
    +    }
    +    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: List[ReceivedBlockTrackerLogEvent]): Unit = {
    +      writeAheadLogOption.foreach { logManager =>
    +        if (records.nonEmpty) {
    +          logDebug(s"Batched ${records.length} records for WAL write")
    +          logManager.write(ByteBuffer.wrap(Utils.serialize(CombinedRBTLogEvent(records))), 
    +            clock.getTimeMillis())
    +        }
    +      }
    +    }
    +
    +    def stop(): Unit = {
    +      active = false
    +    }
    +
    +    private def flushRecords(): Unit = {
    +      val buffer = new ArrayBuffer[ReceivedBlockTrackerLogEvent]()
    +      while (!walWriteQueue.isEmpty) {
    +        buffer.append(walWriteQueue.poll())
    +      }
    +      val records = buffer.toList
    +      try {
    +        writeRecords(records)
    +        records.foreach(walWriteStatusMap.put(_, Success))
    +      } catch {
    +        case e: Exception =>
    +          logWarning(s"Batch WAL Writer failed to write $records")
    +          records.foreach(walWriteStatusMap.put(_, Fail))
    +      }
    +    }
    +
    +    override def run(): Unit = {
    +      while (active) {
    +        flushRecords()
    +        Thread.sleep(WAL_WRITE_STATUS_CHECK_BACKOFF)
    --- End diff --
    
    If you can change `walWriteQueue` to `LinkedBlockingQueue`, you can use `take` to wait for new records rather than using a while loop of `Thread.sleep`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155129707
  
    Added the test for abrupt close, and added the batching to manual writer


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43452841
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala ---
    @@ -64,6 +65,10 @@ private[streaming] object WriteAheadLogUtils extends Logging {
         }
       }
     
    +  def isBatchingEnabled(conf: SparkConf): Boolean = {
    --- End diff --
    
    check `isDriver()` here.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154861724
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151734788
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44234566
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +493,28 @@ 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)) {
    +          val f = Future {
    +            synchronized {
    +              if (active) {
    +                addBlock(receivedBlockInfo)
    +              } else {
    +                throw new IllegalStateException("Receiver Tracker Endpoint shutdown.")
    +              }
    +            }
    +          }(walBatchingThreadPool)
    +          f.onComplete { result =>
    +            synchronized {
    +              if (active) {
    +                context.reply(result.get)
    +              } else {
    +                context.reply(new IllegalStateException("Receiver Tracker Endpoint shutdown."))
    --- End diff --
    
    You are returning an exception here ... are you sure that the other side can receiver exceptions at all? I think it only receives boolean - https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala#L161



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44086282
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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
    --- End diff --
    
    Yes, @burak can you give a short example explaining the why the clean logic fails. In fact, put that in the class doc, and refer to it from here. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44183084
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,187 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    -
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    -
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
    +      }
    +    )
    +    promise
       }
     
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    +  test("BatchedWriteAheadLog - records get added to a queue") {
    +    val numSuccess = new AtomicInteger()
    +    val numFail = new AtomicInteger()
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL)
    +
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    // make sure queue is empty initially
    +    assert(wal.getQueueLength === 0)
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 5L, numSuccess, numFail)
    +    eventFuture(wal, event2, 10L, numSuccess, numFail)
    +    eventFuture(wal, event3, 11L, numSuccess, numFail)
    +    eventFuture(wal, event4, 12L, numSuccess, numFail)
    +    eventFuture(wal, event5, 20L, numSuccess, numFail)
    +
    +    eventually(Eventually.timeout(2 seconds)) {
    +      // the first element will immediately be taken and the rest will get queued
    --- End diff --
    
    This is not guaranteed. `BatchedWriteAheadLog.flushRecords` just drains all records in the queue, and the queue length could be anything. You may not be able to observe 4.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44236366
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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.{SparkConf, SparkException, Logging}
    +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.
    + *
    + * In addition, notice that the write method is still a blocking call. This will ensure that a
    + * receiver will not be able to submit multiple `AddBlock` calls, jeopardizing the ordering of data.
    + *
    + * 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 SparkException("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 SparkException("close() was called on BatchedWriteAheadLog before " +
    +        s"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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    --- End diff --
    
    Can you name this thread?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154352789
  
    **[Test build #45208 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45208/consoleFull)** for PR 9143 at commit [`f3f79dd`](https://github.com/apache/spark/commit/f3f79dda27cb35b22d00ff5709571ec26f1cab6f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972317
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala ---
    @@ -103,13 +112,6 @@ class ReceivedBlockTrackerSuite
           manualClock.advance(timeIncrementMillis)
         }
     
    -    // Generate and add blocks to the given tracker
    --- End diff --
    
    Why move this section of the code


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155226728
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152042908
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44234476
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +493,28 @@ 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)) {
    +          val f = Future {
    +            synchronized {
    +              if (active) {
    +                addBlock(receivedBlockInfo)
    +              } else {
    +                throw new IllegalStateException("Receiver Tracker Endpoint shutdown.")
    +              }
    +            }
    +          }(walBatchingThreadPool)
    +          f.onComplete { result =>
    +            synchronized {
    --- End diff --
    
    Why is this synchronized necessary?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44085972
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    Are you thinking that the Scala iterator wrapper is putting all the objects from the Java iterator into some list and then exposing as a scala iterator from the list? No!! That would just defeat the purpose of the conversion, and no one would be using this in Scala (scala is pretty mature man :). 
    
    `JavaIterator.asScala` just allocates one and only one extra object - a wrapper object that wraps a Java iterator into a scala iterator, and forwards the next, and hasNext of the scala iterator to the corresponding functions of the Java iterator. That's it. If you want to see the code, search of `JIteratorWrapper` in [here](http://xuwei-k.github.io/scala-library-sxr/scala-library-2.10.0/scala/collection/convert/Wrappers.scala.html#undefined).
    
    And the vice versa for `ScalaIterator.asJava`. So there is no list in the middle, there is not collecting in a intermediate list, there is no buffering, it is lazy all the way through. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42721281
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -477,7 +493,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
               registerReceiver(streamId, typ, hostPort, receiverEndpoint, context.senderAddress)
             context.reply(successful)
           case AddBlock(receivedBlockInfo) =>
    -        context.reply(addBlock(receivedBlockInfo))
    +        if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf)) {
    +          val f = Future(addBlockAsync(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    +        } else {
    +          context.reply(addBlock(receivedBlockInfo))
    --- End diff --
    
    I think we should also put this line to `Future` since `addBlock` may be slow.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152042833
  
    **[Test build #44550 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44550/consoleFull)** for PR 9143 at commit [`13cab44`](https://github.com/apache/spark/commit/13cab44bcd1ed4f5c1fa5bb855375f137fa15768).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154871903
  
    **[Test build #45317 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45317/consoleFull)** for PR 9143 at commit [`2c6d515`](https://github.com/apache/spark/commit/2c6d5154339b772a6cce66696f12d65e39c2779e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44176718
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * 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.{SparkException, Logging}
    +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.
    + *
    + * All other methods of the WriteAheadLog interface will be passed on to the wrapped WriteAheadLog.
    + */
    +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  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]()
    +    walWriteQueue.offer(Record(byteBuffer, time, promise))
    +    Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
    +  }
    +
    +  /**
    +   * 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 = {
    +    wrappedLog.read(segment)
    +  }
    +
    +  /**
    +   * 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("BatchedWriteAheadLog shutting down.")
    +    active = false
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    while (!walWriteQueue.isEmpty) {
    +      val Record(_, _, promise) = walWriteQueue.poll()
    +      promise.failure(new SparkException("close() was called on BatchedWriteAheadLog."))
    +    }
    +    wrappedLog.close()
    --- End diff --
    
    There is a race condition here. `BatchedWriteAheadLog.write` may be called when we reach here, then a `Record` will be put into walWriteQueue and swallowed. How about the following fix:
    ```Scala
      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, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
        } else {
          throw new SparkException("close() was called on BatchedWriteAheadLog.")
        }
      }
    
      override def close(): Unit = {
        logInfo("BatchedWriteAheadLog shutting down.")
        synchronized {
          active = false
        }
        batchedWriterThread.interrupt()
        batchedWriterThread.join()
        while (!walWriteQueue.isEmpty) {
          val Record(_, _, promise) = walWriteQueue.poll()
          promise.failure(new SparkException("close() was called on BatchedWriteAheadLog."))
        }
        wrappedLog.close()
      }
    ```
    
    BTW, it's better to add `time` in the exception message for better debug.



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153913349
  
    **[Test build #45069 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45069/consoleFull)** for PR 9143 at commit [`c0063d3`](https://github.com/apache/spark/commit/c0063d3b5c611e85091ec86c8b6ee355c982cb21).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155240999
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154861718
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44236068
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +493,28 @@ 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)) {
    +          val f = Future {
    +            synchronized {
    +              if (active) {
    +                addBlock(receivedBlockInfo)
    +              } else {
    +                throw new IllegalStateException("Receiver Tracker Endpoint shutdown.")
    +              }
    +            }
    +          }(walBatchingThreadPool)
    +          f.onComplete { result =>
    +            synchronized {
    --- End diff --
    
    Yeah. That's' a valid race condition. Its safer and simpler to reason about if `active` is made `@volatile`. You can removed the synchronized in line 507.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152031956
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44086176
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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])
    --- End diff --
    
    Agreed. It can be even simpler, just "Record". And the `record` field can be called `data`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42717743
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -40,7 +42,8 @@ private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks:
       extends ReceivedBlockTrackerLogEvent
     private[streaming] case class BatchCleanupEvent(times: Seq[Time])
       extends ReceivedBlockTrackerLogEvent
    -
    +private[streaming] case class CombinedRBTLogEvent(events: List[ReceivedBlockTrackerLogEvent])
    --- End diff --
    
    nit: I prefer to use `CombinedReceivedBlockTrackerLogEvent` since we don't use `RBT` in other places and the meaning of `RBT` is not obvious.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154930726
  
    **[Test build #45341 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45341/consoleFull)** for PR 9143 at commit [`c659cd7`](https://github.com/apache/spark/commit/c659cd7cfd9ef011f16583752aea4fcd244c3e02).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `  case class Record(data: ByteBuffer, time: Long, promise: Promise[WriteAheadLogRecordHandle])`\n


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43040224
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -122,6 +161,26 @@ private[streaming] class ReceivedBlockTracker(
         }
       }
     
    +  /**
    +   * Allocate all unallocated blocks to the given batch.
    +   * This event will get written to the write ahead log (if enabled).
    +   */
    +  def allocateBlocksToBatch(batchTime: Time): Unit = {
    --- End diff --
    
    Why have three versions of every func - e.g. allocateBlocksToBatch, allocateBlocksToBatchAync, and allocateBlocksToBatch0? Why not have a single version - allocateBlocksToBatch, which takes an optional parameter async: Boolean. It will call writeToLog with that parameter, and writeToLog can check that async parameter to decide what to do.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152277305
  
    **[Test build #44619 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44619/consoleFull)** for PR 9143 at commit [`ea62d25`](https://github.com/apache/spark/commit/ea62d253f7e8d33484ab19263a7b61968b774d67).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152022682
  
    @tdas @zsxwing I've been running recovery tests with this patch and everything seems to be working so far. Could you please take another look?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44239702
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    -  }
    -
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    -    }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
           }
    -    }
    +    )
    +    promise
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    --- End diff --
    
    Why will 3 be automatically flushed?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151734773
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154561544
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154544421
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155148707
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151749320
  
    Merged build finished. Test PASSed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155226632
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43962050
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -439,6 +439,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
         private val submitJobThreadPool = ExecutionContext.fromExecutorService(
           ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool"))
     
    +    private val walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonCachedThreadPool("wal-batching-thead-pool"))
    --- End diff --
    
    "thread-pool"


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44239889
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,283 +281,150 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    -
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with PrivateMethodTester {
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    -  }
    +  private var fileBasedWAL: FileBasedWriteAheadLog = _
    +  private var walHandle: FileBasedWriteAheadLogSegment = _
    +  private var walBatchingThreadPool: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    +  override def beforeEach(): Unit = {
    +    fileBasedWAL = mock[FileBasedWriteAheadLog]
    +    walHandle = mock[FileBasedWriteAheadLogSegment]
    +    walBatchingThreadPool = ExecutionContext.fromExecutorService(
    +      ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool"))
       }
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    -
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  override def afterEach(): Unit = {
    +    if (walBatchingThreadPool != null) {
    +      walBatchingThreadPool.shutdownNow()
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    -
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    -
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    -    }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
    -  }
    -
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    -
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    assert(deaggregate.toSeq === events)
       }
    -}
     
    -object WriteAheadLogSuite {
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(fileBasedWAL.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
     
    -  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 = { }
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      wal.write(buffer, 2L)
    +    }
       }
     
    -  class MockWriteAheadLog1(val conf: SparkConf) extends MockWriteAheadLog0()
    -
    -  class MockWriteAheadLog2(val conf: SparkConf, x: Int) extends MockWriteAheadLog0()
    -
    -
    -  private val hadoopConf = new Configuration()
    -
    -  /** Write data to a file directly and return an array of the file segments written. */
    -  def writeDataManually(data: Seq[String], file: String): Seq[FileBasedWriteAheadLogSegment] = {
    -    val segments = new ArrayBuffer[FileBasedWriteAheadLogSegment]()
    -    val writer = HdfsUtils.getOutputStream(file, hadoopConf)
    -    data.foreach { item =>
    -      val offset = writer.getPos
    -      val bytes = Utils.serialize(item)
    -      writer.writeInt(bytes.size)
    -      writer.write(bytes)
    -      segments += FileBasedWriteAheadLogSegment(file, offset, bytes.size)
    -    }
    -    writer.close()
    -    segments
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def eventFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingThreadPool)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingThreadPool)
       }
     
       /**
    -   * Write data to a file using the writer class and return an array of the file segments written.
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
        */
    -  def writeDataUsingWriter(
    -      filePath: String,
    -      data: Seq[String]
    -    ): Seq[FileBasedWriteAheadLogSegment] = {
    -    val writer = new FileBasedWriteAheadLogWriter(filePath, hadoopConf)
    -    val segments = data.map {
    -      item => writer.write(item)
    -    }
    -    writer.close()
    -    segments
    -  }
    -
    -  /** Write data to rotating files in log directory using the WriteAheadLog class. */
    -  def writeDataUsingWriteAheadLog(
    -      logDirectory: String,
    -      data: Seq[String],
    -      manualClock: ManualClock = new ManualClock,
    -      closeLog: Boolean = true,
    -      clockAdvanceTime: Int = 500,
    -      closeFileAfterWrite: Boolean = false): FileBasedWriteAheadLog = {
    -    if (manualClock.getTimeMillis() < 100000) manualClock.setTime(10000)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1,
    -      closeFileAfterWrite)
    -
    -    // Ensure that 500 does not get sorted after 2000, so put a high base value.
    -    data.foreach { item =>
    -      manualClock.advance(clockAdvanceTime)
    -      wal.write(item, manualClock.getTimeMillis())
    -    }
    -    if (closeLog) wal.close()
    -    wal
    -  }
    -
    -  /** Read data from a segments of a log file directly and return the list of byte buffers. */
    -  def readDataManually(segments: Seq[FileBasedWriteAheadLogSegment]): Seq[String] = {
    -    segments.map { segment =>
    -      val reader = HdfsUtils.getInputStream(segment.path, hadoopConf)
    -      try {
    -        reader.seek(segment.offset)
    -        val bytes = new Array[Byte](segment.length)
    -        reader.readInt()
    -        reader.readFully(bytes)
    -        val data = Utils.deserialize[String](bytes)
    -        reader.close()
    -        data
    -      } finally {
    -        reader.close()
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[FileBasedWriteAheadLogSegment] {
    +        override def answer(invocation: InvocationOnMock): FileBasedWriteAheadLogSegment = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
           }
    -    }
    +    )
    +    promise
       }
     
    -  /** Read all the data from a log file directly and return the list of byte buffers. */
    -  def readDataManually(file: String): Seq[String] = {
    -    val reader = HdfsUtils.getInputStream(file, hadoopConf)
    -    val buffer = new ArrayBuffer[String]
    -    try {
    -      while (true) {
    -        // Read till EOF is thrown
    -        val length = reader.readInt()
    -        val bytes = new Array[Byte](length)
    -        reader.read(bytes)
    -        buffer += Utils.deserialize[String](bytes)
    -      }
    -    } catch {
    -      case ex: EOFException =>
    -    } finally {
    -      reader.close()
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val wal = new BatchedWriteAheadLog(fileBasedWAL, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(fileBasedWAL)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    eventFuture(wal, event1, 3L) // 3 will automatically be flushed for the first write
    +    // rest of the records will be batched while it takes 3 to get written
    +    eventFuture(wal, event2, 5L)
    +    eventFuture(wal, event3, 8L)
    +    eventFuture(wal, event4, 12L)
    +    eventFuture(wal, event5, 10L)
    +    promise.success(true)
    --- End diff --
    
    Are you sure that all the futures have been already blocked before you set promise to true?  I think you need another check to make sure that all the threads are actually blocked. You could check the number of threads active in the `walBatchingThreadPool` in an `eventually` for that. And later check them again to make sure that all the threads have been released.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153416277
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154262705
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45183/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43330013
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -195,30 +236,44 @@ private[streaming] class ReceivedBlockTracker(
           timeToAllocatedBlocks --= batchTimes
         }
     
    +    def resolveEvent(event: ReceivedBlockTrackerLogEvent): Unit = {
    +      event match {
    +        case CombinedReceivedBlockTrackerLogEvent(events) =>
    +          events.foreach(resolveEvent)
    +        case BlockAdditionEvent(receivedBlockInfo) =>
    +          insertAddedBlock(receivedBlockInfo)
    +        case BatchAllocationEvent(time, allocatedBlocks) =>
    +          insertAllocatedBatch(time, allocatedBlocks)
    +        case BatchCleanupEvent(batchTimes) =>
    +          cleanupBatches(batchTimes)
    +      }
    +    }
    +
         writeAheadLogOption.foreach { writeAheadLog =>
           logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}")
           writeAheadLog.readAll().asScala.foreach { byteBuffer =>
             logTrace("Recovering record " + byteBuffer)
    -        Utils.deserialize[ReceivedBlockTrackerLogEvent](
    -          byteBuffer.array, Thread.currentThread().getContextClassLoader) match {
    -          case BlockAdditionEvent(receivedBlockInfo) =>
    -            insertAddedBlock(receivedBlockInfo)
    -          case BatchAllocationEvent(time, allocatedBlocks) =>
    -            insertAllocatedBatch(time, allocatedBlocks)
    -          case BatchCleanupEvent(batchTimes) =>
    -            cleanupBatches(batchTimes)
    -        }
    +        resolveEvent(Utils.deserialize[ReceivedBlockTrackerLogEvent](
    +          byteBuffer.array, Thread.currentThread().getContextClassLoader))
           }
         }
       }
     
       /** Write an update to the tracker to the write ahead log */
    -  private def writeToLog(record: ReceivedBlockTrackerLogEvent) {
    -    if (isWriteAheadLogEnabled) {
    -      logDebug(s"Writing to log $record")
    +  private[streaming] def writeToLog(record: ReceivedBlockTrackerLogEvent): Boolean = {
    +    if (!isWriteAheadLogEnabled) return true
    +    if (WriteAheadLogUtils.isBatchingEnabled(conf)) {
    +      val promise = Promise[Boolean]()
    +      walWriteStatusMap.put(record, promise)
    +      walWriteQueue.offer(record)
    +      Await.result(promise.future.recover { case _ => false }(batchWriterThreadPool),
    --- End diff --
    
    we need to add a try-catch block with `TimeoutException` here which returns `false`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43809439
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +491,12 @@ 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)) {
    +          val f = Future(addBlock(receivedBlockInfo))(walBatchingThreadPool)
    +          f.onComplete(result => context.reply(result.get))(walBatchingThreadPool)
    --- End diff --
    
    @zsxwing, with the reference to the context still be the same if done lazily?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154522500
  
    Overall, pretty good. Just some corder cases during `close`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44236326
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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.{SparkConf, SparkException, Logging}
    +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.
    + *
    + * In addition, notice that the write method is still a blocking call. This will ensure that a
    + * receiver will not be able to submit multiple `AddBlock` calls, jeopardizing the ordering of data.
    --- End diff --
    
    nit: Make this not refer to the receiver. You can say the following.
    
    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.



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154258376
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148562637
  
      [Test build #43823 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43823/consoleFull) for   PR 9143 at commit [`cf1ad3d`](https://github.com/apache/spark/commit/cf1ad3d2c21c9019259793661020a3f29d1bca25).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44090581
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,135 +362,197 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    -
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +/** Tests for the aggregation, deaggregation related methods in the BatchedWriteAheadLog object */
    +class BatchedWriteAheadLogUtilsSuite extends SparkFunSuite {
    --- End diff --
    
    It just tests the `aggregate`, `deaggregate` methods. There's no need to run them twice with `closeFileAfterWrite` on and off


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42724272
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -428,6 +441,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
         private val submitJobThreadPool = ExecutionContext.fromExecutorService(
           ThreadUtils.newDaemonCachedThreadPool("submit-job-thead-pool"))
     
    +    private val walBatchingThreadPool = ExecutionContext.fromExecutorService(
    --- End diff --
    
    It's better to shutdown this thread pool in `onStop`. If `addBlockAsync` is very slow, we may exhaust threads because the number of `block` could be a big number. How about making `addBlockAsync` become real asynchronous. The `addBlockAsync` would become
    
    ```
    def addBlockAsync(event): Future[...] = {
       put the event to queue with a Promise
       promise.future.mapTo { v => 
           do some work after writing 
       }
    }
    ```


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43961907
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -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))
    --- End diff --
    
    This is not actually new, but I think we should clean first and then remove the times from the `timeToAllocatedBlocks` map. Otherwise, failed cleanups can lead to the logs never getting cleaned up at all.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43451237
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -176,7 +201,7 @@ private[streaming] class ReceivedBlockTracker(
         def insertAddedBlock(receivedBlockInfo: ReceivedBlockInfo) {
           logTrace(s"Recovery: Inserting added block $receivedBlockInfo")
           receivedBlockInfo.setBlockIdInvalid()
    -      getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo
    +      afterBlockAddAcknowledged(receivedBlockInfo)
    --- End diff --
    
    I think its fine to keep this as is and not create a separate method for this.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43813962
  
    --- 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.
    --- End diff --
    
    Why not throw exception for failure?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44087186
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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)
    --- End diff --
    
    It is weird to signify failure by saying `promise.success(null)`. Rather better to fail the promise by passing on different reasons, and let the promise-waiting-code handle the failure. Otherwise, there are multiple places that are signifying failures using `promise.success`, and then there is a `promise.recover` which is handle failures elsewhere ... not very intuitive.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42719354
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +350,67 @@ 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 =>
    +      new Thread(runnable, "Batch WAL Writer").start()
    +      Runtime.getRuntime.addShutdownHook(new Thread {
    --- End diff --
    
    Could you remove this shutdown hook? If you find a case that`ReceivedBlockTracker` won't be stopped, it's better to fix it rather than adding a shutdown hook.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43056920
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala ---
    @@ -36,6 +36,7 @@ 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.enableBatching"
    --- End diff --
    
    maybe rename this to `spark.streaming.driver.writeAheadLog.allowBatching`, just in case we have add a config `driver.wal.enable` (similar to `receiver.wal.enable`) in future.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154259691
  
    **[Test build #45183 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45183/consoleFull)** for PR 9143 at commit [`ec6d8da`](https://github.com/apache/spark/commit/ec6d8da236c6a742ae523a81ebe4754013dc4a59).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44243764
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,223 @@
    +/*
    + * 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 NonFatal(e) =>
    +        logWarning(s"BatchedWriteAheadLog Writer failed to write $buffer", e)
    +        buffer.foreach(_.promise.failure(e))
    +    } finally {
    +      buffer.clear()
    +    }
    +  }
    +
    +  /** Method for querying the queue length during tests. */
    +  private[util] def getQueueLength(): Int = walWriteQueue.size()
    --- End diff --
    
    ermm.. same problem again.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153913866
  
    **[Test build #45069 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45069/consoleFull)** for PR 9143 at commit [`c0063d3`](https://github.com/apache/spark/commit/c0063d3b5c611e85091ec86c8b6ee355c982cb21).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155250816
  
    Yeah does not make sense for it to be in there.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148563170
  
    Merged build finished. Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43964017
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    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(): 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("Batched WAL Writer thread exiting.")
    +      }
    +    }, "Batched WAL Writer")
    +    thread.setDaemon(true)
    +    thread.start()
    +    thread
    +  }
    +
    +  /** 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
    --- End diff --
    
    Is the `take` being done to ensure that there is at least 1 record being written out? Isn't it better to just better to call `drainTo` and check if `numBatched > 0` ? That seems far more intuitive.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-153913870
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45069/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43973123
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -102,6 +100,180 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
       }
     
    +  test("WriteAheadLogUtils - wrap WriteAheadLog in BatchedWriteAheadLog when batching is enabled") {
    +    def getBatchedSparkConf: SparkConf =
    +      new SparkConf().set("spark.streaming.driver.writeAheadLog.allowBatching", "true")
    +
    +    val emptyConf = getBatchedSparkConf
    +    assertDriverLogClass[FileBasedWriteAheadLog](emptyConf, isBatched = true)
    +    assertReceiverLogClass[FileBasedWriteAheadLog](emptyConf)
    +
    +    // Verify setting driver WAL class
    +    val conf1 = getBatchedSparkConf.set("spark.streaming.driver.writeAheadLog.class",
    +      classOf[MockWriteAheadLog0].getName())
    +    assertDriverLogClass[MockWriteAheadLog0](conf1, isBatched = true)
    +    assertReceiverLogClass[FileBasedWriteAheadLog](conf1)
    +
    +    // 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)
    +  }
    +}
    +
    +/** Common tests for WriteAheadLogs that we would like to test with different configurations. */
    +abstract class CommonWriteAheadLogTests(
    +    allowBatching: Boolean,
    +    closeFileAfterWrite: Boolean,
    +    testTag: String = "")
    +  extends SparkFunSuite with BeforeAndAfter {
    +
    +  import WriteAheadLogSuite._
    +
    +  val hadoopConf = new Configuration()
    +  var tempDir: File = null
    +  var testDir: String = null
    +  var testFile: String = null
    +  var writeAheadLog: WriteAheadLog = null
    +  protected def testPrefix = if (testTag != "") testTag + " - " else testTag
    +
    +  before {
    +    tempDir = Utils.createTempDir()
    +    testDir = tempDir.toString
    +    testFile = new File(tempDir, "testFile").toString
    +    if (writeAheadLog != null) {
    +      writeAheadLog.close()
    +      writeAheadLog = null
    +    }
    +  }
    +
    +  after {
    +    Utils.deleteRecursively(tempDir)
    +  }
    +
    +  test(testPrefix + "read rotating logs") {
    +    // Write data manually for testing reading through WriteAheadLog
    +    val writtenData = (1 to 10).map { i =>
    +      val data = generateRandomData()
    +      val file = testDir + s"/log-$i-$i"
    +      writeDataManually(data, file)
    +      data
    +    }.flatten
    +
    +    val logDirectoryPath = new Path(testDir)
    +    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    +    assert(fileSystem.exists(logDirectoryPath) === true)
    +
    +    // Read data using manager and verify
    +    val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching)
    +    assert(readData === writtenData)
    +  }
    +
    +  test(testPrefix + "write rotating logs") {
    --- End diff --
    
    `write`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44236121
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +493,28 @@ 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)) {
    +          val f = Future {
    +            synchronized {
    +              if (active) {
    --- End diff --
    
    You are calling the whole `addBlock` in `synchronized`. Isnt that what we wanted to avoid originally, so that the method does not block other threads why the WAL is being written (that why we removed synchronized inside the whole method of addBlock and made it very specific on a few lines only)? 



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43972801
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -19,59 +19,57 @@ package org.apache.spark.streaming.util
     import java.io._
     import java.nio.ByteBuffer
     import java.util
    +import java.util.concurrent.atomic.AtomicInteger
     
     import scala.collection.JavaConverters._
     import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise, Future, ExecutionContext}
     import scala.concurrent.duration._
     import scala.language.{implicitConversions, postfixOps}
     import scala.reflect.ClassTag
     
     import org.apache.hadoop.conf.Configuration
     import org.apache.hadoop.fs.Path
    +import org.mockito.Matchers._
    +import org.mockito.Matchers.{eq => meq}
    +import org.mockito.Mockito._
    +import org.mockito.invocation.InvocationOnMock
    +import org.mockito.stubbing.Answer
    +import org.scalatest.concurrent.Eventually
     import org.scalatest.concurrent.Eventually._
     import org.scalatest.BeforeAndAfter
    +import org.scalatest.mock.MockitoSugar.{mock => mmock}
     
    -import org.apache.spark.util.{ManualClock, Utils}
    +import org.apache.spark.streaming.scheduler._
    +import org.apache.spark.util.{ThreadUtils, ManualClock, Utils}
     import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
     
    -class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
    -
    +class WriteAheadLogUtilsSuite extends SparkFunSuite {
    --- End diff --
    
    Could you pull this class out into a new file? Along with all the needed classes like `MockWAL...`. This file has become a huge one, and better to split it as much as we can. 


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148563602
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43821/
    Test FAILed.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44240396
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -58,49 +71,127 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         Utils.deleteRecursively(tempDir)
       }
     
    -  test("WriteAheadLogUtils - log selection and creation") {
    -    val logDir = Utils.createTempDir().getAbsolutePath()
    +  test(testPrefix + "read all logs") {
    +    // Write data manually for testing reading through WriteAheadLog
    +    val writtenData = (1 to 10).map { i =>
    +      val data = generateRandomData()
    +      val file = testDir + s"/log-$i-$i"
    +      writeDataManually(data, file)
    --- End diff --
    
    It's very important that the BatchedWAL can recover when the existing data is not batched as well, right? I mean, isn't it possible that some users start their stream using Spark 1.5. Then with the release, they want to use the BatchedWAL, therefore they enable it. All the existing data in the checkpoint directory is not batched, but the BatchedWAL should be able to recover from it still


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155241026
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-150147496
  
    My 2 cents about this PR: is it possible to create a new `WriteAheadLog` (such as, BatchWriteAheadLog) that wraps the existing `WriteAheadLog` and implement the new logic in `BatchWriteAheadLog`? Then you don't need to add so many pairs of `XXX` and `XXXAsync`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r42719556
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +350,67 @@ 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 =>
    +      new Thread(runnable, "Batch WAL Writer").start()
    --- End diff --
    
    It's better to set this thread to daemon.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44244198
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala ---
    @@ -190,149 +284,165 @@ class WriteAheadLogSuite extends SparkFunSuite with BeforeAndAfter {
         }
         reader.close()
       }
    +}
     
    -  test("FileBasedWriteAheadLog - write rotating logs") {
    -    // Write data with rotation using WriteAheadLog class
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -
    -    // Read data manually to verify the written data
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    -    assert(writtenData === dataToWrite)
    -  }
    +abstract class CloseFileAfterWriteTests(allowBatching: Boolean, testTag: String)
    +  extends CommonWriteAheadLogTests(allowBatching, closeFileAfterWrite = true, testTag) {
     
    -  test("FileBasedWriteAheadLog - close after write flag") {
    +  import WriteAheadLogSuite._
    +  test(testPrefix + "close after write flag") {
         // Write data with rotation using WriteAheadLog class
         val numFiles = 3
         val dataToWrite = Seq.tabulate(numFiles)(_.toString)
         // total advance time is less than 1000, therefore log shouldn't be rolled, but manually closed
         writeDataUsingWriteAheadLog(testDir, dataToWrite, closeLog = false, clockAdvanceTime = 100,
    -      closeFileAfterWrite = true)
    +      closeFileAfterWrite = true, allowBatching = allowBatching)
     
         // Read data manually to verify the written data
         val logFiles = getLogFilesInDirectory(testDir)
         assert(logFiles.size === numFiles)
    -    val writtenData = logFiles.flatMap { file => readDataManually(file)}
    +    val writtenData: Seq[String] = readAndDeserializeDataManually(logFiles, allowBatching)
         assert(writtenData === dataToWrite)
       }
    +}
     
    -  test("FileBasedWriteAheadLog - read rotating logs") {
    -    // Write data manually for testing reading through WriteAheadLog
    -    val writtenData = (1 to 10).map { i =>
    -      val data = generateRandomData()
    -      val file = testDir + s"/log-$i-$i"
    -      writeDataManually(data, file)
    -      data
    -    }.flatten
    +class FileBasedWriteAheadLogWithFileCloseAfterWriteSuite
    +  extends CloseFileAfterWriteTests(allowBatching = false, "FileBasedWriteAheadLog")
     
    -    val logDirectoryPath = new Path(testDir)
    -    val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
    -    assert(fileSystem.exists(logDirectoryPath) === true)
    +class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests(
    +    allowBatching = true,
    +    closeFileAfterWrite = false,
    +    "BatchedWriteAheadLog") with MockitoSugar with BeforeAndAfterEach with Eventually {
     
    -    // Read data using manager and verify
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === writtenData)
    -  }
    +  import BatchedWriteAheadLog._
    +  import WriteAheadLogSuite._
     
    -  test("FileBasedWriteAheadLog - recover past logs when creating new manager") {
    -    // Write data with manager, recover with new manager and verify
    -    val dataToWrite = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(dataToWrite === readData)
    +  private var wal: WriteAheadLog = _
    +  private var walHandle: WriteAheadLogRecordHandle = _
    +  private var walBatchingThreadPool: ThreadPoolExecutor = _
    +  private var walBatchingExecutionContext: ExecutionContextExecutorService = _
    +  private val sparkConf = new SparkConf()
    +
    +  override def beforeEach(): Unit = {
    +    wal = mock[WriteAheadLog]
    +    walHandle = mock[WriteAheadLogRecordHandle]
    +    walBatchingThreadPool = ThreadUtils.newDaemonFixedThreadPool(8, "wal-test-thread-pool")
    +    walBatchingExecutionContext = ExecutionContext.fromExecutorService(walBatchingThreadPool)
       }
     
    -  test("FileBasedWriteAheadLog - clean old logs") {
    -    logCleanUpTest(waitForCompletion = false)
    +  override def afterEach(): Unit = {
    +    if (walBatchingExecutionContext != null) {
    +      walBatchingExecutionContext.shutdownNow()
    +    }
       }
     
    -  test("FileBasedWriteAheadLog - clean old logs synchronously") {
    -    logCleanUpTest(waitForCompletion = true)
    -  }
    +  test("BatchedWriteAheadLog - serializing and deserializing batched records") {
    +    val events = Seq(
    +      BlockAdditionEvent(ReceivedBlockInfo(0, None, None, null)),
    +      BatchAllocationEvent(null, null),
    +      BatchCleanupEvent(Nil)
    +    )
     
    -  private def logCleanUpTest(waitForCompletion: Boolean): Unit = {
    -    // Write data with manager, recover with new manager and verify
    -    val manualClock = new ManualClock
    -    val dataToWrite = generateRandomData()
    -    writeAheadLog = writeDataUsingWriteAheadLog(testDir, dataToWrite, manualClock, closeLog = false)
    -    val logFiles = getLogFilesInDirectory(testDir)
    -    assert(logFiles.size > 1)
    +    val buffers = events.map(e => Record(ByteBuffer.wrap(Utils.serialize(e)), 0L, null))
    +    val batched = BatchedWriteAheadLog.aggregate(buffers)
    +    val deaggregate = BatchedWriteAheadLog.deaggregate(batched).map(buffer =>
    +      Utils.deserialize[ReceivedBlockTrackerLogEvent](buffer.array()))
     
    -    writeAheadLog.clean(manualClock.getTimeMillis() / 2, waitForCompletion)
    +    assert(deaggregate.toSeq === events)
    +  }
     
    -    if (waitForCompletion) {
    -      assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -    } else {
    -      eventually(timeout(1 second), interval(10 milliseconds)) {
    -        assert(getLogFilesInDirectory(testDir).size < logFiles.size)
    -      }
    +  test("BatchedWriteAheadLog - failures in wrappedLog get bubbled up") {
    +    when(wal.write(any[ByteBuffer], anyLong)).thenThrow(new RuntimeException("Hello!"))
    +    // the BatchedWriteAheadLog should bubble up any exceptions that may have happened during writes
    +    val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)
    +
    +    intercept[RuntimeException] {
    +      val buffer = mock[ByteBuffer]
    +      batchedWal.write(buffer, 2L)
         }
       }
     
    -  test("FileBasedWriteAheadLog - handling file errors while reading rotating logs") {
    -    // Generate a set of log files
    -    val manualClock = new ManualClock
    -    val dataToWrite1 = generateRandomData()
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite1, manualClock)
    -    val logFiles1 = getLogFilesInDirectory(testDir)
    -    assert(logFiles1.size > 1)
    +  // we make the write requests in separate threads so that we don't block the test thread
    +  private def writeEventWithFuture(
    +      wal: WriteAheadLog,
    +      event: String,
    +      time: Long,
    +      numSuccess: AtomicInteger = null,
    +      numFail: AtomicInteger = null): Unit = {
    +    val f = Future(wal.write(event, time))(walBatchingExecutionContext)
    +    f.onComplete {
    +      case Success(v) =>
    +        assert(v === walHandle) // return our mock handle after the write
    +        if (numSuccess != null) numSuccess.incrementAndGet()
    +      case Failure(v) => if (numFail != null) numFail.incrementAndGet()
    +    }(walBatchingExecutionContext)
    +  }
     
    +  /**
    +   * In order to block the writes on the writer thread, we mock the write method, and block it
    +   * for some time with a promise.
    +   */
    +  private def writeBlockingPromise(wal: WriteAheadLog): Promise[Any] = {
    +    // we would like to block the write so that we can queue requests
    +    val promise = Promise[Any]()
    +    when(wal.write(any[ByteBuffer], any[Long])).thenAnswer(
    +      new Answer[WriteAheadLogRecordHandle] {
    +        override def answer(invocation: InvocationOnMock): WriteAheadLogRecordHandle = {
    +          Await.ready(promise.future, 4.seconds)
    +          walHandle
    +        }
    +      }
    +    )
    +    promise
    +  }
     
    -    // Recover old files and generate a second set of log files
    -    val dataToWrite2 = generateRandomData()
    -    manualClock.advance(100000)
    -    writeDataUsingWriteAheadLog(testDir, dataToWrite2, manualClock)
    -    val logFiles2 = getLogFilesInDirectory(testDir)
    -    assert(logFiles2.size > logFiles1.size)
    +  test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") {
    +    val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)
    +    // block the write so that we can batch some records
    +    val promise = writeBlockingPromise(wal)
    +
    +    val event1 = "hello"
    +    val event2 = "world"
    +    val event3 = "this"
    +    val event4 = "is"
    +    val event5 = "doge"
    +
    +    // 3 will automatically be flushed for the first write
    +    writeEventWithFuture(batchedWal, event1, 3L)
    +    // rest of the records will be batched while it takes 3 to get written
    +    writeEventWithFuture(batchedWal, event2, 5L)
    +    writeEventWithFuture(batchedWal, event3, 8L)
    +    writeEventWithFuture(batchedWal, event4, 12L)
    +    writeEventWithFuture(batchedWal, event5, 10L)
    +    eventually(timeout(1 second)) {
    +      assert(walBatchingThreadPool.getActiveCount === 5)
    +    }
    +    promise.success(true)
     
    -    // Read the files and verify that all the written data can be read
    -    val readData1 = readDataUsingWriteAheadLog(testDir)
    -    assert(readData1 === (dataToWrite1 ++ dataToWrite2))
    +    val buffer1 = wrapArrayArrayByte(Array(event1))
    +    val buffer2 = wrapArrayArrayByte(Array(event2, event3, event4, event5))
     
    -    // Corrupt the first set of files so that they are basically unreadable
    -    logFiles1.foreach { f =>
    -      val raf = new FileOutputStream(f, true).getChannel()
    -      raf.truncate(1)
    -      raf.close()
    +    eventually(timeout(1 second)) {
    +      verify(wal, times(1)).write(meq(buffer1), meq(3L))
    +      // the file name should be the timestamp of the last record, as events should be naturally
    +      // in order of timestamp, and we need the last element.
    +      verify(wal, times(1)).write(meq(buffer2), meq(10L))
         }
    -
    -    // Verify that the corrupted files do not prevent reading of the second set of data
    -    val readData = readDataUsingWriteAheadLog(testDir)
    -    assert(readData === dataToWrite2)
       }
     
    -  test("FileBasedWriteAheadLog - do not create directories or files unless write") {
    -    val nonexistentTempPath = File.createTempFile("test", "")
    -    nonexistentTempPath.delete()
    -    assert(!nonexistentTempPath.exists())
    +  test("BatchedWriteAheadLog - shutdown properly") {
    +    val batchedWal = new BatchedWriteAheadLog(wal, sparkConf)
    +    batchedWal.close()
    +    verify(wal, times(1)).close()
     
    -    val writtenSegment = writeDataManually(generateRandomData(), testFile)
    -    val wal = new FileBasedWriteAheadLog(new SparkConf(), tempDir.getAbsolutePath,
    -      new Configuration(), 1, 1, closeFileAfterWrite = false)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by creating log object")
    -    wal.read(writtenSegment.head)
    -    assert(!nonexistentTempPath.exists(), "Directory created just by attempting to read segment")
    +    intercept[IllegalStateException](batchedWal.write(mock[ByteBuffer], 12L))
    --- End diff --
    
    this is good but not sufficient. sorry if I wasnt clear earlier. I want to make sure that we test the following behavior
    
    - batched wal active
    - write1, write2 concurrently waiting to complete
    - batched wal stopped, wrapped wal asked to stop
    - but right before the wrapped wal stops, the write of the wrapped wal completes
    - the 2 threads waiting for writes to complete should get exceptions, instead of getting success or blocked indefinitely
    
    If they get success after the system has been asked to stop, that may lead to hard-to-reason cases about data loss. So better to test that this behavior does not break.
    



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152639101
  
    **[Test build #44702 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44702/consoleFull)** for PR 9143 at commit [`81e7f93`](https://github.com/apache/spark/commit/81e7f93ee67c31ea55f21808868ebf3081e54782).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155204978
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155213853
  
    **[Test build #45423 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45423/consoleFull)** for PR 9143 at commit [`7ec55c5`](https://github.com/apache/spark/commit/7ec55c5f637a61a75e54456dfded5fe663c16a04).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152291565
  
    **[Test build #44627 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44627/consoleFull)** for PR 9143 at commit [`e288cb2`](https://github.com/apache/spark/commit/e288cb2382d32ced61e7ae6f81a3b7cca2815eae).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-155226351
  
    LGTM, will merge when tests pass.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154342369
  
     Merged build triggered.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152637690
  
    Merged build started.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-148563585
  
      [Test build #43821 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/43821/console) for   PR 9143 at commit [`ee36f89`](https://github.com/apache/spark/commit/ee36f8968354ecb14f1153b9bd7fb8f0d4bb9e1e).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43330116
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala ---
    @@ -235,8 +290,85 @@ 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())
    +        while (!walWriteQueue.isEmpty) {
    --- End diff --
    
    will change this to `walWriteQueue.drainTo(buffer)` instead


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44070214
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    +    batchedWriterThread.interrupt()
    +    batchedWriterThread.join()
    +    fulfillPromises()
    +    parent.close()
    +  }
    +
    +  /**
    +   * Respond to any promises that may have been left in the queue, to unblock receivers during
    +   * shutdown.
    +   */
    +  private def fulfillPromises(): Unit = {
    --- End diff --
    
    Can we call this something else? This really is not fulfilling the promise - it more like, for lack of another word, breaking the promise. Maybe call this `unblockReceivers`?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44092490
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    --- End diff --
    
    It has a single asJava call. We use the beauty of methods like Seq.map, tail, everywhere else. I think this should stay as an ArrayBuffer. Too much looping overhead otherwise


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154342609
  
    **[Test build #45208 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/45208/consoleFull)** for PR 9143 at commit [`f3f79dd`](https://github.com/apache/spark/commit/f3f79dda27cb35b22d00ff5709571ec26f1cab6f).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44057628
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.
    + *
    + * Parent exposed for testing.
    + */
    +private[streaming] class BatchedWriteAheadLog(private[util] val parent: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  protected val walWriteQueue = new LinkedBlockingQueue[RecordBuffer]()
    +
    +  private val WAL_WRITE_STATUS_TIMEOUT = 5000 // 5 seconds
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  protected val buffer = new ArrayBuffer[RecordBuffer]()
    +
    +  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]()
    +    walWriteQueue.offer(RecordBuffer(byteBuffer, time, promise))
    +    try {
    +      Await.result(promise.future.recover { case _ => null }(ThreadUtils.sameThread),
    +        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
    --- End diff --
    
    `flatMap` does not, but the `asScala` and `asJava` calls are allocating new objects. They maybe lazily collecting the objects being iterated over, but they are still creating two new objects no? In a loop, the only new object being created is the list itself. The others are basically references to already created objects returned from the `deaggregate()` call.
    
    It might be more code, but I never found loops hard to read - sometimes I find loops clearer. 
    
    I don't feel that strongly about it - I just feel that a couple of scala to java (and vice versa) conversions just for a `flatMap()` call is not really worth it. Being Young-gen GC they are not that bad, I guess.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-154529683
  
    Just got a small question about the patch: Now if a receiver sends two ReceivedBlockInfo A and B, we may add block in the reverse order B and A to `receivedBlockTracker`. Does it break any assumption in current code base?


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43814263
  
    --- 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
    --- End diff --
    
    You need to keep a reference to the `Batched WAL Writer` thread and interrupt it here. Otherwise, it may hang at `walWriteQueue.take()`.


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43811813
  
    --- 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] = {
    +    try {
    +      Utils.deserialize[Array[Array[Byte]]](buffer.array()).map(ByteBuffer.wrap)
    --- End diff --
    
    See my previous comment about `buffer.array()`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by brkyvz <gi...@git.apache.org>.
Github user brkyvz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r44234779
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala ---
    @@ -488,7 +493,28 @@ 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)) {
    +          val f = Future {
    +            synchronized {
    +              if (active) {
    +                addBlock(receivedBlockInfo)
    +              } else {
    +                throw new IllegalStateException("Receiver Tracker Endpoint shutdown.")
    +              }
    +            }
    +          }(walBatchingThreadPool)
    +          f.onComplete { result =>
    +            synchronized {
    --- End diff --
    
    Per the race condition between @zsxwing and Hari


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-151735132
  
    **[Test build #44502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44502/consoleFull)** for PR 9143 at commit [`80a0b8d`](https://github.com/apache/spark/commit/80a0b8d9e994ccf5c9381e12dae4c736ad6c3800).


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/9143#issuecomment-152051139
  
    Discussed offline. Its cleaner to refactor this. Have a BatchedWritedAheadLog which is a wrapper for other WALs, and is a WAL itself. Internally it will take care of queueing and batching, and recovering data from batches. This will be easily unit testable. 
    
    The RBT then does not need to change at all, only wrap the WAL in BatchedWAL when configured. 



---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43435260
  
    --- 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"))
    --- End diff --
    
    shouldnt this be a `newDaemonFixedThreadPool(nThreads = 1, ...)`


---
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


[GitHub] spark pull request: [SPARK-11141][STREAMING] Batch ReceivedBlockTr...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/9143#discussion_r43811146
  
    --- 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 = {
    --- End diff --
    
    Same comment as below.


---
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