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

    https://github.com/apache/spark/pull/3026#discussion_r19696931
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
 ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.scheduler
    +
    +import java.nio.ByteBuffer
    +
    +import scala.collection.mutable
    +import scala.language.implicitConversions
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.Path
    +
    +import org.apache.spark.{Logging, SparkConf}
    +import org.apache.spark.streaming.Time
    +import org.apache.spark.streaming.util.{Clock, WriteAheadLogManager}
    +import org.apache.spark.util.Utils
    +
    +/** Trait representing any event in the ReceivedBlockTracker that updates 
its state. */
    +private[streaming] sealed trait ReceivedBlockTrackerLogEvent
    +
    +private[streaming] case class BlockAdditionEvent(receivedBlockInfo: 
ReceivedBlockInfo)
    +  extends ReceivedBlockTrackerLogEvent
    +private[streaming] case class BatchAllocationEvent(time: Time, 
allocatedBlocks: AllocatedBlocks)
    +  extends ReceivedBlockTrackerLogEvent
    +private[streaming] case class BatchCleanupEvent(times: Seq[Time])
    +  extends ReceivedBlockTrackerLogEvent
    +
    +
    +/** Class representing the blocks of all the streams allocated to a batch 
*/
    +private[streaming] case class AllocatedBlocks(streamIdToAllocatedBlocks: 
Map[Int, Seq[ReceivedBlockInfo]]) {
    +  def getBlockForStream(streamId: Int) = 
streamIdToAllocatedBlocks(streamId)
    +}
    +
    +/**
    + * Class that keep track of all the received blocks, and allocate them to 
batches
    + * when required. All actions taken by this class can be saved to a write 
ahead log
    + * (if a checkpoint directory has been provided), so that the state of the 
tracker
    + * (received blocks and block-to-batch allocations) can be recovered after 
driver failure.
    + *
    + * Note that when any instance of this class is created with a checkpoint 
directory,
    + * it will try reading events from logs in the directory.
    + */
    +private[streaming] class ReceivedBlockTracker(
    +    conf: SparkConf,
    +    hadoopConf: Configuration,
    +    streamIds: Seq[Int],
    +    clock: Clock,
    +    checkpointDirOption: Option[String])
    +  extends Logging {
    +
    +  private type ReceivedBlockQueue = mutable.Queue[ReceivedBlockInfo]
    +  
    +  private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, 
ReceivedBlockQueue]
    +  private val timeToAllocatedBlocks = new mutable.HashMap[Time, 
AllocatedBlocks]
    +
    +  private val logManagerRollingIntervalSecs = conf.getInt(
    +    
"spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", 60)
    +  private val logManagerOption = checkpointDirOption.map { checkpointDir =>
    +    new WriteAheadLogManager(
    +      ReceivedBlockTracker.checkpointDirToLogDir(checkpointDir),
    +      hadoopConf,
    +      rollingIntervalSecs = logManagerRollingIntervalSecs,
    +      callerName = "ReceivedBlockHandlerMaster",
    +      clock = clock
    +    )
    +  }
    +
    +  // Recover block information from write ahead logs
    +  recoverFromWriteAheadLogs()
    +
    +  /** Add received block. This event will get written to the write ahead 
log (if enabled). */
    +  def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = 
synchronized {
    +    try {
    +      writeToLog(BlockAdditionEvent(receivedBlockInfo))
    +      getReceivedBlockQueue(receivedBlockInfo.streamId) += 
receivedBlockInfo
    +      logDebug(s"Stream ${receivedBlockInfo.streamId} received " +
    +        s"block ${receivedBlockInfo.blockStoreResult.blockId}")
    +      true
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error adding block $receivedBlockInfo", e)
    +        false
    +    }
    +  }
    +
    +  /**
    +   * 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 {
    +    val allocatedBlocks = AllocatedBlocks(streamIds.map { streamId =>
    --- End diff --
    
    This is a really dense expression. Can this be broken out into simpler 
experesssions that make it easier to read?
    
    ```
    val streamsWithBlocks = streamIds.map { streamId =>
          (streamId, getReceivedBlockQueue(streamId).dequeueAll(_ => true))
        }
        val streamToBlocks = streamsWithBlocks.toMap
        val allocatedBlocks = AllocatedBlocks(streamToBlocks)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to