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

    https://github.com/apache/spark/pull/2882#discussion_r19318248
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.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 scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{ExecutionContext, Future}
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.Path
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.spark.Logging
    +import org.apache.spark.util.Utils
    +import WriteAheadLogManager._
    +
    +/**
    + * This class manages write ahead log files.
    + * - Writes records (bytebuffers) to periodically rotating log files.
    + * - Recovers the log files and the reads the recovered records upon 
failures.
    + * - Cleans up old log files.
    + *
    + * Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write
    + * and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read.
    + *
    + *@param logDirectory Directory when rotating log files will be created.
    + * @param hadoopConf Hadoop configuration for reading/writing log files.
    + * @param rollingIntervalSecs The interval in seconds with which logs will 
be rolled over.
    + *                            Default is one minute.
    + * @param maxFailures Max number of failures that is tolerated for every 
attempt to write to log.
    + *                    Default is three.
    + * @param callerName Optional name of the class who is using this manager.
    + * @param clock Optional clock that is used to check for rotation interval.
    + */
    +private[streaming] class WriteAheadLogManager(
    +    logDirectory: String,
    +    hadoopConf: Configuration,
    +    rollingIntervalSecs: Int = 60,
    +    maxFailures: Int = 3,
    +    callerName: String = "",
    +    clock: Clock = new SystemClock
    +  ) extends Logging {
    +
    +  private val pastLogs = new ArrayBuffer[LogInfo]
    +  private val callerNameTag =
    +    if (callerName != null && callerName.nonEmpty) s" for $callerName" 
else ""
    +  private val threadpoolName = s"WriteAheadLogManager $callerNameTag"
    +  implicit private val executionContext = 
ExecutionContext.fromExecutorService(
    +    Utils.newDaemonFixedThreadPool(1, threadpoolName))
    +  override protected val logName = s"WriteAheadLogManager $callerNameTag"
    +
    +  private var currentLogPath: String = null
    +  private var currentLogWriter: WriteAheadLogWriter = null
    +  private var currentLogWriterStartTime: Long = -1L
    +  private var currentLogWriterStopTime: Long = -1L
    +
    +  initializeOrRecover()
    +
    +  /** Write a byte buffer to the log file */
    +  def writeToLog(byteBuffer: ByteBuffer): FileSegment = synchronized {
    +    var fileSegment: FileSegment = null
    +    var failures = 0
    +    var lastException: Exception = null
    +    var succeeded = false
    +    while (!succeeded && failures < maxFailures) {
    +      try {
    +        fileSegment = getLogWriter(clock.currentTime).write(byteBuffer)
    +        succeeded = true
    +      } catch {
    +        case ex: Exception =>
    +          lastException = ex
    +          logWarning("Failed to write to write ahead log")
    +          resetWriter()
    +          failures += 1
    +      }
    +    }
    +    if (fileSegment == null) {
    +      logError(s"Failed to write to write ahead log after $failures 
failures")
    +      throw lastException
    +    }
    +    fileSegment
    +  }
    +
    +  /**
    +   * Read all the existing logs from the log directory.
    +   *
    +   * Note that this is typically called when the caller is initializing 
and wants
    +   * to recover past  state from the write ahead logs (that is, before 
making any writes).
    +   * If this is called after writes have been made using this manager, 
then it may not return
    +   * the latest the records. This does not deal with currently active log 
files, and
    +   * hence the implementation is kept simple.
    +   */
    +  def readFromLog(): Iterator[ByteBuffer] = synchronized {
    +    val logFilesToRead = pastLogs.map{ _.path} ++ Option(currentLogPath)
    +    logInfo("Reading from the logs: " + logFilesToRead.mkString("\n"))
    +    logFilesToRead.iterator.map { file =>
    +        logDebug(s"Creating log reader with $file")
    +        new WriteAheadLogReader(file, hadoopConf)
    +    } flatMap { x => x }
    --- End diff --
    
    Ah, that makes sense, actually: `flatMap(identity)` on an iterator here is 
the equivalent of Python's itertools.chain, whereas `flatten` might imply 
materializing the flattened collection, which doesn't make sense for an 
iterator.


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