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

    https://github.com/apache/spark/pull/3026#discussion_r19653425
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala
 ---
    @@ -0,0 +1,207 @@
    +/*
    + * 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.storage.StreamBlockId
    +import org.apache.spark.streaming.Time
    +import org.apache.spark.streaming.util.{Clock, WriteAheadLogManager}
    +import org.apache.spark.util.Utils
    +
    +/** Trait representing any action done in the ReceivedBlockTracker */
    +private[streaming] sealed trait ReceivedBlockTrackerAction
    +
    +private[streaming] case class BlockAddition(receivedBlockInfo: 
ReceivedBlockInfo)
    +  extends ReceivedBlockTrackerAction
    +private[streaming] case class BatchAllocations(time: Time, 
allocatedBlocks: AllocatedBlocks)
    +  extends ReceivedBlockTrackerAction
    +private[streaming] case class BatchCleanup(times: Seq[Time])
    +  extends ReceivedBlockTrackerAction
    +
    +
    +/** Class representing the blocks of all the streams allocated to a batch 
*/
    +case class AllocatedBlocks(streamIdToAllocatedBlocks: Map[Int, 
Seq[ReceivedBlockInfo]]) {
    +  def apply(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,
    + * so that the state of the tracker (received blocks and block-to-batch 
allocations)
    + * can be recovered after driver failure.
    + */
    --- End diff --
    
    It would be good to explain that as a side effect of construction this will 
attempt to read any existing state in the log contained in the checkpoint 
directory. Alternatively, to keep the constructor side effect free, what if the 
caller explicitly called `recoverFromWriteAheadLogs` when it constructs this 
(and you'd throw an error if someone called that when there were already 
blocks).


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