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

    https://github.com/apache/spark/pull/5645#discussion_r29309292
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala
 ---
    @@ -0,0 +1,129 @@
    +/*
    + * 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 scala.util.control.NonFatal
    +
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.util.Utils
    +import org.apache.spark.{Logging, SparkConf, SparkException}
    +
    +/** A helper class with utility functions related to the WriteAheadLog 
interface */
    +private[streaming] object WriteAheadLogUtils extends Logging {
    +  val RECEIVER_WAL_ENABLE_CONF_KEY = 
"spark.streaming.receiver.writeAheadLog.enable"
    +  val RECEIVER_WAL_CLASS_CONF_KEY = 
"spark.streaming.receiver.writeAheadLog.class"
    +  val RECEIVER_WAL_ROLLING_INTERVAL_CONF_KEY =
    +    "spark.streaming.receiver.writeAheadLog.rollingIntervalSecs"
    +  val RECEIVER_WAL_MAX_FAILURES_CONF_KEY = 
"spark.streaming.receiver.writeAheadLog.maxFailures"
    +
    +  val DRIVER_WAL_CLASS_CONF_KEY = 
"spark.streaming.driver.writeAheadLog.class"
    +  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 DEFAULT_ROLLING_INTERVAL_SECS = 60
    +  val DEFAULT_MAX_FAILURES = 3
    +
    +  def enableReceiverLog(conf: SparkConf): Boolean = {
    +    conf.getBoolean(RECEIVER_WAL_ENABLE_CONF_KEY, false)
    +  }
    +
    +  def getRollingIntervalSecs(conf: SparkConf, isDriver: Boolean): Int = {
    +    if (isDriver) {
    +      conf.getInt(DRIVER_WAL_ROLLING_INTERVAL_CONF_KEY, 
DEFAULT_ROLLING_INTERVAL_SECS)
    +    } else {
    +      conf.getInt(RECEIVER_WAL_ROLLING_INTERVAL_CONF_KEY, 
DEFAULT_ROLLING_INTERVAL_SECS)
    +    }
    +  }
    +
    +  def getMaxFailures(conf: SparkConf, isDriver: Boolean): Int = {
    +    if (isDriver) {
    +      conf.getInt(DRIVER_WAL_MAX_FAILURES_CONF_KEY, DEFAULT_MAX_FAILURES)
    +    } else {
    +      conf.getInt(RECEIVER_WAL_MAX_FAILURES_CONF_KEY, DEFAULT_MAX_FAILURES)
    +    }
    +  }
    +
    +  /**
    +   * Create a WriteAheadLog for the driver. If configured with custom WAL 
class, it will try
    +   * to create instance of that class, otherwise it will create the 
default FileBasedWriteAheadLog.
    +   */
    +  def createLogForDriver(
    +      sparkConf: SparkConf,
    +      fileWalLogDirectory: String,
    +      fileWalHadoopConf: Configuration
    +    ): WriteAheadLog = {
    +    createLog(true, sparkConf, fileWalLogDirectory, fileWalHadoopConf)
    +  }
    +
    +  /**
    +   * Create a WriteAheadLog for the receiver. If configured with custom 
WAL class, it will try
    +   * to create instance of that class, otherwise it will create the 
default FileBasedWriteAheadLog.
    +   */
    +  def createLogForReceiver(
    +      sparkConf: SparkConf,
    +      fileWalLogDirectory: String,
    +      fileWalHadoopConf: Configuration
    +    ): WriteAheadLog = {
    +    createLog(false, sparkConf, fileWalLogDirectory, fileWalHadoopConf)
    +  }
    +
    +  /**
    +   * Create a WriteAheadLog based on the value of the given config key. 
The config key is used
    +   * to get the class name from the SparkConf. If the class is configured, 
it will try to
    +   * create instance of that class by first trying `new 
CustomWAL(sparkConf, logDir)` then trying
    +   * `new CustomWAL(sparkConf)`. If either fails, it will fail. If no 
class is configured, then
    +   * it will create the default FileBasedWriteAheadLog.
    +   */
    +  private def createLog(
    +      isDriver: Boolean,
    +      sparkConf: SparkConf,
    +      fileWalLogDirectory: String,
    +      fileWalHadoopConf: Configuration
    +    ): WriteAheadLog = {
    +
    +    val classNameOption = if (isDriver) {
    +      sparkConf.getOption(DRIVER_WAL_CLASS_CONF_KEY)
    +    } else {
    +      sparkConf.getOption(RECEIVER_WAL_CLASS_CONF_KEY)
    +    }
    +    classNameOption.map { className =>
    +      try {
    +        instantiateClass(
    +          
Utils.classForName(className).asInstanceOf[Class[WriteAheadLog]], sparkConf)
    +      } catch {
    +        case NonFatal(e) =>
    +          throw new SparkException(s"Could not create a write ahead log of 
class $className", e)
    +      }
    +    }.getOrElse {
    +      new FileBasedWriteAheadLog(sparkConf, fileWalLogDirectory, 
fileWalHadoopConf,
    +        getRollingIntervalSecs(sparkConf, isDriver), 
getMaxFailures(sparkConf, isDriver))
    +    }
    +  }
    +
    +  /** Instantiate the class, either using single arg constructor or zero 
arg constructor */
    +  private def instantiateClass(cls: Class[WriteAheadLog], conf: 
SparkConf): WriteAheadLog = {
    --- End diff --
    
    I think `Class[WriteAheadLog]` should be changed to `Class[_ <: 
WriteAheadLog]`.


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