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

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

Reply via email to