aboutsummaryrefslogtreecommitdiff
path: root/streaming
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2014-01-06 03:05:52 -0800
committerTathagata Das <tathagata.das1565@gmail.com>2014-01-06 03:05:52 -0800
commit3b4c4c7f4d0d6e45a1acb0baf0d9416a8997b686 (patch)
tree98b744beacdb06925eba8413288c473b13f49174 /streaming
parentd0fd3b9ad238294346eb3465c489eabd41fb2380 (diff)
parenta2e7e0497484554f86bd71e93705eb0422b1512b (diff)
downloadspark-3b4c4c7f4d0d6e45a1acb0baf0d9416a8997b686.tar.gz
spark-3b4c4c7f4d0d6e45a1acb0baf0d9416a8997b686.tar.bz2
spark-3b4c4c7f4d0d6e45a1acb0baf0d9416a8997b686.zip
Merge remote-tracking branch 'apache/master' into project-refactor
Conflicts: examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
Diffstat (limited to 'streaming')
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala66
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala1
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala13
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala89
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala18
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala38
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala153
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala8
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala9
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala16
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala71
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala4
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java4
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala8
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala59
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala15
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala21
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala5
21 files changed, 377 insertions, 234 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 7b343d2376..ca0115f90e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -21,12 +21,13 @@ import java.io._
import java.util.concurrent.Executors
import java.util.concurrent.RejectedExecutionException
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.conf.Configuration
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.deploy.SparkHadoopUtil
private[streaming]
@@ -34,14 +35,14 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
extends Logging with Serializable {
val master = ssc.sc.master
val framework = ssc.sc.appName
- val sparkHome = ssc.sc.sparkHome
+ val sparkHome = ssc.sc.getSparkHome.getOrElse(null)
val jars = ssc.sc.jars
- val environment = ssc.sc.environment
val graph = ssc.graph
val checkpointDir = ssc.checkpointDir
val checkpointDuration = ssc.checkpointDuration
val pendingTimes = ssc.scheduler.getPendingTimes()
- val delaySeconds = MetadataCleaner.getDelaySeconds
+ val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
+ val sparkConf = ssc.conf
def validate() {
assert(master != null, "Checkpoint.master is null")
@@ -54,36 +55,36 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
/**
- * Convenience class to speed up the writing of graph checkpoint to file
+ * Convenience class to handle the writing of graph checkpoint to file
*/
private[streaming]
-class CheckpointWriter(checkpointDir: String) extends Logging {
+class CheckpointWriter(conf: SparkConf, checkpointDir: String, hadoopConf: Configuration)
+ extends Logging
+{
val file = new Path(checkpointDir, "graph")
- // The file to which we actually write - and then "move" to file.
- private val writeFile = new Path(file.getParent, file.getName + ".next")
- private val bakFile = new Path(file.getParent, file.getName + ".bk")
-
- private var stopped = false
-
- val conf = new Configuration()
- var fs = file.getFileSystem(conf)
- val maxAttempts = 3
+ val MAX_ATTEMPTS = 3
val executor = Executors.newFixedThreadPool(1)
+ val compressionCodec = CompressionCodec.createCodec(conf)
+ // The file to which we actually write - and then "move" to file
+ val writeFile = new Path(file.getParent, file.getName + ".next")
+ // The file to which existing checkpoint is backed up (i.e. "moved")
+ val bakFile = new Path(file.getParent, file.getName + ".bk")
- private val compressionCodec = CompressionCodec.createCodec()
+ private var stopped = false
+ private var fs_ : FileSystem = _
- // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
+ // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
// I did not notice any errors - reintroduce it ?
-
class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
def run() {
var attempts = 0
val startTime = System.currentTimeMillis()
- while (attempts < maxAttempts) {
+ while (attempts < MAX_ATTEMPTS && !stopped) {
attempts += 1
try {
logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
- // This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast.
+ // This is inherently thread unsafe, so alleviating it by writing to '.new' and
+ // then moving it to the final file
val fos = fs.create(writeFile)
fos.write(bytes)
fos.close()
@@ -101,6 +102,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
} catch {
case ioe: IOException =>
logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
+ reset()
}
}
logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'")
@@ -133,7 +135,17 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
val startTime = System.currentTimeMillis()
val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS)
val endTime = System.currentTimeMillis()
- logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.")
+ logInfo("CheckpointWriter executor terminated ? " + terminated +
+ ", waited for " + (endTime - startTime) + " ms.")
+ }
+
+ private def fs = synchronized {
+ if (fs_ == null) fs_ = file.getFileSystem(hadoopConf)
+ fs_
+ }
+
+ private def reset() = synchronized {
+ fs_ = null
}
}
@@ -141,11 +153,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
private[streaming]
object CheckpointReader extends Logging {
- def read(path: String): Checkpoint = {
+ def read(conf: SparkConf, path: String): Checkpoint = {
val fs = new Path(path).getFileSystem(new Configuration())
- val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+ val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"),
+ new Path(path), new Path(path + ".bk"))
- val compressionCodec = CompressionCodec.createCodec()
+ val compressionCodec = CompressionCodec.createCodec(conf)
attempts.foreach(file => {
if (fs.exists(file)) {
@@ -158,7 +171,8 @@ object CheckpointReader extends Logging {
// loader to find and load classes. This is a well know Java issue and has popped up
// in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
val zis = compressionCodec.compressedInputStream(fis)
- val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
+ val ois = new ObjectInputStreamWithLoader(zis,
+ Thread.currentThread().getContextClassLoader)
val cp = ois.readObject.asInstanceOf[Checkpoint]
ois.close()
fs.close()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index a78d3965ee..00671ba520 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -56,8 +56,6 @@ abstract class DStream[T: ClassTag] (
@transient protected[streaming] var ssc: StreamingContext
) extends Serializable with Logging {
- initLogging()
-
// =======================================================================
// Methods that should be implemented by subclasses of DStream
// =======================================================================
@@ -208,7 +206,7 @@ abstract class DStream[T: ClassTag] (
checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
)
- val metadataCleanerDelay = MetadataCleaner.getDelaySeconds
+ val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf)
logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
assert(
metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index daed7ff7c3..a09b891956 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -24,7 +24,6 @@ import org.apache.spark.Logging
import org.apache.spark.streaming.scheduler.Job
final private[streaming] class DStreamGraph extends Serializable with Logging {
- initLogging()
private val inputStreams = new ArrayBuffer[InputDStream[_]]()
private val outputStreams = new ArrayBuffer[DStream[_]]()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 80af96c060..56dbcbda23 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -108,8 +108,9 @@ extends Serializable {
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
- partitioner: Partitioner) : DStream[(K, C)] = {
- new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
+ partitioner: Partitioner,
+ mapSideCombine: Boolean = true): DStream[(K, C)] = {
+ new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine)
}
/**
@@ -173,7 +174,13 @@ extends Serializable {
slideDuration: Duration,
partitioner: Partitioner
): DStream[(K, Seq[V])] = {
- self.window(windowDuration, slideDuration).groupByKey(partitioner)
+ val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v
+ val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v
+ val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2
+ self.groupByKey(partitioner)
+ .window(windowDuration, slideDuration)
+ .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner)
+ .asInstanceOf[DStream[(K, Seq[V])]]
}
/**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 96f57cbcc8..693cb7fc30 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -23,7 +23,6 @@ import scala.reflect.ClassTag
import java.io.InputStream
import java.util.concurrent.atomic.AtomicInteger
-import java.util.UUID
import akka.actor.Props
import akka.actor.SupervisorStrategy
@@ -41,7 +40,6 @@ import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receivers._
import org.apache.spark.streaming.scheduler._
-
/**
* A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
* information (such as, cluster URL and job name) to internally create a SparkContext, it provides
@@ -55,18 +53,27 @@ class StreamingContext private (
/**
* Create a StreamingContext using an existing SparkContext.
- * @param sparkContext Existing SparkContext
- * @param batchDuration The time interval at which streaming data will be divided into batches
+ * @param sparkContext existing SparkContext
+ * @param batchDuration the time interval at which streaming data will be divided into batches
*/
def this(sparkContext: SparkContext, batchDuration: Duration) = {
this(sparkContext, null, batchDuration)
}
/**
+ * Create a StreamingContext by providing the configuration necessary for a new SparkContext.
+ * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+ * @param batchDuration the time interval at which streaming data will be divided into batches
+ */
+ def this(conf: SparkConf, batchDuration: Duration) = {
+ this(StreamingContext.createNewSparkContext(conf), null, batchDuration)
+ }
+
+ /**
* Create a StreamingContext by providing the details necessary for creating a new SparkContext.
- * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param appName A name for your job, to display on the cluster web UI
- * @param batchDuration The time interval at which streaming data will be divided into batches
+ * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param appName a name for your job, to display on the cluster web UI
+ * @param batchDuration the time interval at which streaming data will be divided into batches
*/
def this(
master: String,
@@ -79,26 +86,25 @@ class StreamingContext private (
null, batchDuration)
}
-
/**
* Re-create a StreamingContext from a checkpoint file.
* @param path Path either to the directory that was specified as the checkpoint directory, or
* to the checkpoint file 'graph' or 'graph.bk'.
*/
- def this(path: String) = this(null, CheckpointReader.read(path), null)
-
- initLogging()
+ def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null)
if (sc_ == null && cp_ == null) {
throw new Exception("Spark Streaming cannot be initialized with " +
"both SparkContext and checkpoint as null")
}
- if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) {
- MetadataCleaner.setDelaySeconds(cp_.delaySeconds)
+ private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf)
+
+ if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) {
+ MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds)
}
- if (MetadataCleaner.getDelaySeconds < 0) {
+ if (MetadataCleaner.getDelaySeconds(conf_) < 0) {
throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
+ "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
}
@@ -107,12 +113,14 @@ class StreamingContext private (
protected[streaming] val sc: SparkContext = {
if (isCheckpointPresent) {
- new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment)
+ new SparkContext(cp_.sparkConf)
} else {
sc_
}
}
+ protected[streaming] val conf = sc.conf
+
protected[streaming] val env = SparkEnv.get
protected[streaming] val graph: DStreamGraph = {
@@ -133,7 +141,7 @@ class StreamingContext private (
protected[streaming] var checkpointDir: String = {
if (isCheckpointPresent) {
- sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true)
+ sc.setCheckpointDir(cp_.checkpointDir)
cp_.checkpointDir
} else {
null
@@ -168,8 +176,12 @@ class StreamingContext private (
*/
def checkpoint(directory: String) {
if (directory != null) {
- sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
- checkpointDir = directory
+ val path = new Path(directory)
+ val fs = path.getFileSystem(sparkContext.hadoopConfiguration)
+ fs.mkdirs(path)
+ val fullPath = fs.getFileStatus(path).getPath().toString
+ sc.setCheckpointDir(fullPath)
+ checkpointDir = fullPath
} else {
checkpointDir = null
}
@@ -276,7 +288,8 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
- * File names starting with . are ignored.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
* @tparam K Key type for reading HDFS file
* @tparam V Value type for reading HDFS file
@@ -295,6 +308,8 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system.
* @param directory HDFS directory to monitor for new file
* @param filter Function to filter paths to process
* @param newFilesOnly Should process only new files and ignore existing files in the directory
@@ -315,7 +330,9 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them as text files (using key as LongWritable, value
- * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * as Text and input format as TextInputFormat). Files must be written to the
+ * monitored directory by "moving" them from another location within the same
+ * file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
*/
def textFileStream(directory: String): DStream[String] = {
@@ -451,18 +468,36 @@ object StreamingContext {
new PairDStreamFunctions[K, V](stream)
}
+ /**
+ * Find the JAR from which a given class was loaded, to make it easy for users to pass
+ * their JARs to SparkContext.
+ */
+ def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls)
+
+ protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = {
+ // Set the default cleaner delay to an hour if not already set.
+ // This should be sufficient for even 1 second batch intervals.
+ val sc = new SparkContext(conf)
+ if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+ MetadataCleaner.setDelaySeconds(sc.conf, 3600)
+ }
+ sc
+ }
+
protected[streaming] def createNewSparkContext(
master: String,
appName: String,
sparkHome: String,
jars: Seq[String],
- environment: Map[String, String]): SparkContext = {
+ environment: Map[String, String]): SparkContext =
+ {
+ val sc = new SparkContext(master, appName, sparkHome, jars, environment)
// Set the default cleaner delay to an hour if not already set.
- // This should be sufficient for even 1 second interval.
- if (MetadataCleaner.getDelaySeconds < 0) {
- MetadataCleaner.setDelaySeconds(3600)
+ // This should be sufficient for even 1 second batch intervals.
+ if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+ MetadataCleaner.setDelaySeconds(sc.conf, 3600)
}
- new SparkContext(master, appName, sparkHome, jars, environment)
+ sc
}
protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
@@ -474,8 +509,4 @@ object StreamingContext {
prefix + "-" + time.milliseconds + "." + suffix
}
}
-
- protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = {
- new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
- }
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index dfd6e27c3e..6c3467d405 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -155,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
/**
* Combine elements of each key in DStream's RDDs using custom function. This is similar to the
- * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
+ * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
* information.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
@@ -169,6 +169,22 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
}
/**
+ * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+ * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
+ * information.
+ */
+ def combineByKey[C](createCombiner: JFunction[V, C],
+ mergeValue: JFunction2[C, V, C],
+ mergeCombiners: JFunction2[C, C, C],
+ partitioner: Partitioner,
+ mapSideCombine: Boolean
+ ): JavaPairDStream[K, C] = {
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
+ dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine)
+ }
+
+ /**
* Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
* `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
* with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index ea4a0fe619..7068f32517 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -17,20 +17,22 @@
package org.apache.spark.streaming.api.java
+
import scala.collection.JavaConversions._
import scala.reflect.ClassTag
import java.io.InputStream
-import java.util.{Map => JMap, List => JList}
+import java.lang.{Integer => JInt}
+import java.util.{List => JList, Map => JMap}
+import akka.actor.{Props, SupervisorStrategy}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
import org.apache.spark.streaming._
import org.apache.spark.streaming.scheduler.StreamingListener
@@ -117,6 +119,14 @@ class JavaStreamingContext(val ssc: StreamingContext) {
this(new StreamingContext(sparkContext.sc, batchDuration))
/**
+ * Creates a StreamingContext using an existing SparkContext.
+ * @param conf A Spark application configuration
+ * @param batchDuration The time interval at which streaming data will be divided into batches
+ */
+ def this(conf: SparkConf, batchDuration: Duration) =
+ this(new StreamingContext(conf, batchDuration))
+
+ /**
* Re-creates a StreamingContext from a checkpoint file.
* @param path Path either to the directory that was specified as the checkpoint directory, or
* to the checkpoint file 'graph' or 'graph.bk'.
@@ -174,9 +184,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
}
/**
- * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them as text files (using key as LongWritable, value
- * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * as Text and input format as TextInputFormat). Files must be written to the
+ * monitored directory by "moving" them from another location within the same
+ * file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
*/
def textFileStream(directory: String): JavaDStream[String] = {
@@ -218,9 +230,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
}
/**
- * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
- * File names starting with . are ignored.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
* @tparam K Key type for reading HDFS file
* @tparam V Value type for reading HDFS file
@@ -466,5 +479,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* Sstops the execution of the streams.
*/
def stop() = ssc.stop()
+}
+object JavaStreamingContext {
+ /**
+ * Find the JAR from which a given class was loaded, to make it easy for users to pass
+ * their JARs to SparkContext.
+ */
+ def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index 39e25239bf..fb9eda8996 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -17,18 +17,17 @@
package org.apache.spark.streaming.dstream
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.UnionRDD
-import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
-
+import java.io.{ObjectInputStream, IOException}
+import scala.collection.mutable.{HashSet, HashMap}
+import scala.reflect.ClassTag
import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
-import scala.collection.mutable.{HashSet, HashMap}
-import scala.reflect.ClassTag
-
-import java.io.{ObjectInputStream, IOException}
private[streaming]
class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag](
@@ -41,8 +40,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
// Latest file mod time seen till any point of time
- private val lastModTimeFiles = new HashSet[String]()
- private var lastModTime = 0L
+ private val prevModTimeFiles = new HashSet[String]()
+ private var prevModTime = 0L
@transient private var path_ : Path = null
@transient private var fs_ : FileSystem = null
@@ -50,11 +49,11 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
override def start() {
if (newFilesOnly) {
- lastModTime = graph.zeroTime.milliseconds
+ prevModTime = graph.zeroTime.milliseconds
} else {
- lastModTime = 0
+ prevModTime = 0
}
- logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
+ logDebug("LastModTime initialized to " + prevModTime + ", new files only = " + newFilesOnly)
}
override def stop() { }
@@ -69,55 +68,22 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
* the previous call.
*/
override def compute(validTime: Time): Option[RDD[(K, V)]] = {
- assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime)
+ assert(validTime.milliseconds >= prevModTime,
+ "Trying to get new files for really old time [" + validTime + " < " + prevModTime + "]")
- // Create the filter for selecting new files
- val newFilter = new PathFilter() {
- // Latest file mod time seen in this round of fetching files and its corresponding files
- var latestModTime = 0L
- val latestModTimeFiles = new HashSet[String]()
-
- def accept(path: Path): Boolean = {
- if (!filter(path)) { // Reject file if it does not satisfy filter
- logDebug("Rejected by filter " + path)
- return false
- } else { // Accept file only if
- val modTime = fs.getFileStatus(path).getModificationTime()
- logDebug("Mod time for " + path + " is " + modTime)
- if (modTime < lastModTime) {
- logDebug("Mod time less than last mod time")
- return false // If the file was created before the last time it was called
- } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) {
- logDebug("Mod time equal to last mod time, but file considered already")
- return false // If the file was created exactly as lastModTime but not reported yet
- } else if (modTime > validTime.milliseconds) {
- logDebug("Mod time more than valid time")
- return false // If the file was created after the time this function call requires
- }
- if (modTime > latestModTime) {
- latestModTime = modTime
- latestModTimeFiles.clear()
- logDebug("Latest mod time updated to " + latestModTime)
- }
- latestModTimeFiles += path.toString
- logDebug("Accepted " + path)
- return true
- }
- }
- }
- logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime)
- val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString)
+ // Find new files
+ val (newFiles, latestModTime, latestModTimeFiles) = findNewFiles(validTime.milliseconds)
logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n"))
if (newFiles.length > 0) {
// Update the modification time and the files processed for that modification time
- if (lastModTime != newFilter.latestModTime) {
- lastModTime = newFilter.latestModTime
- lastModTimeFiles.clear()
+ if (prevModTime < latestModTime) {
+ prevModTime = latestModTime
+ prevModTimeFiles.clear()
}
- lastModTimeFiles ++= newFilter.latestModTimeFiles
- logDebug("Last mod time updated to " + lastModTime)
+ prevModTimeFiles ++= latestModTimeFiles
+ logDebug("Last mod time updated to " + prevModTime)
}
- files += ((validTime, newFiles))
+ files += ((validTime, newFiles.toArray))
Some(filesToRDD(newFiles))
}
@@ -132,12 +98,28 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n"))
}
+ /**
+ * Find files which have modification timestamp <= current time and return a 3-tuple of
+ * (new files found, latest modification time among them, files with latest modification time)
+ */
+ private def findNewFiles(currentTime: Long): (Seq[String], Long, Seq[String]) = {
+ logDebug("Trying to get new files for time " + currentTime)
+ val filter = new CustomPathFilter(currentTime)
+ val newFiles = fs.listStatus(path, filter).map(_.getPath.toString)
+ (newFiles, filter.latestModTime, filter.latestModTimeFiles.toSeq)
+ }
+
/** Generate one RDD from an array of files */
- protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
- new UnionRDD(
- context.sparkContext,
- files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
- )
+ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
+ val fileRDDs = files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
+ files.zip(fileRDDs).foreach { case (file, rdd) => {
+ if (rdd.partitions.size == 0) {
+ logError("File " + file + " has no data in it. Spark Streaming can only ingest " +
+ "files that have been \"moved\" to the directory assigned to the file stream. " +
+ "Refer to the streaming programming guide for more details.")
+ }
+ }}
+ new UnionRDD(context.sparkContext, fileRDDs)
}
private def path: Path = {
@@ -150,6 +132,10 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
fs_
}
+ private def reset() {
+ fs_ = null
+ }
+
@throws(classOf[IOException])
private def readObject(ois: ObjectInputStream) {
logDebug(this.getClass().getSimpleName + ".readObject used")
@@ -191,6 +177,51 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas
hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]"
}
}
+
+ /**
+ * Custom PathFilter class to find new files that have modification timestamps <= current time,
+ * but have not been seen before (i.e. the file should not be in lastModTimeFiles)
+ */
+ private[streaming]
+ class CustomPathFilter(maxModTime: Long) extends PathFilter {
+ // Latest file mod time seen in this round of fetching files and its corresponding files
+ var latestModTime = 0L
+ val latestModTimeFiles = new HashSet[String]()
+
+ def accept(path: Path): Boolean = {
+ try {
+ if (!filter(path)) { // Reject file if it does not satisfy filter
+ logDebug("Rejected by filter " + path)
+ return false
+ }
+ val modTime = fs.getFileStatus(path).getModificationTime()
+ logDebug("Mod time for " + path + " is " + modTime)
+ if (modTime < prevModTime) {
+ logDebug("Mod time less than last mod time")
+ return false // If the file was created before the last time it was called
+ } else if (modTime == prevModTime && prevModTimeFiles.contains(path.toString)) {
+ logDebug("Mod time equal to last mod time, but file considered already")
+ return false // If the file was created exactly as lastModTime but not reported yet
+ } else if (modTime > maxModTime) {
+ logDebug("Mod time more than ")
+ return false // If the file is too new that considering it may give errors
+ }
+ if (modTime > latestModTime) {
+ latestModTime = modTime
+ latestModTimeFiles.clear()
+ logDebug("Latest mod time updated to " + latestModTime)
+ }
+ latestModTimeFiles += path.toString
+ logDebug("Accepted " + path)
+ } catch {
+ case fnfe: java.io.FileNotFoundException =>
+ logWarning("Error finding new files", fnfe)
+ reset()
+ return false
+ }
+ return true
+ }
+ }
}
private[streaming]
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 5add20871e..27d474c0a0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -88,8 +88,6 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe
*/
abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging {
- initLogging()
-
lazy protected val env = SparkEnv.get
lazy protected val actor = env.actorSystem.actorOf(
@@ -176,8 +174,8 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
/** A helper actor that communicates with the NetworkInputTracker */
private class NetworkReceiverActor extends Actor {
logInfo("Attempting to register with tracker")
- val ip = System.getProperty("spark.driver.host", "localhost")
- val port = System.getProperty("spark.driver.port", "7077").toInt
+ val ip = env.conf.get("spark.driver.host", "localhost")
+ val port = env.conf.get("spark.driver.port", "7077").toInt
val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
val tracker = env.actorSystem.actorSelection(url)
val timeout = 5.seconds
@@ -214,7 +212,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging
case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
val clock = new SystemClock()
- val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong
+ val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong
val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
val blockStorageLevel = storageLevel
val blocksForPushing = new ArrayBlockingQueue[Block](1000)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
index e6e0022097..84e69f277b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
@@ -29,8 +29,9 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
- partitioner: Partitioner
- ) extends DStream [(K,C)] (parent.ssc) {
+ partitioner: Partitioner,
+ mapSideCombine: Boolean = true
+ ) extends DStream[(K,C)] (parent.ssc) {
override def dependencies = List(parent)
@@ -38,8 +39,8 @@ class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
override def compute(validTime: Time): Option[RDD[(K,C)]] = {
parent.getOrCompute(validTime) match {
- case Some(rdd) =>
- Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
+ case Some(rdd) => Some(rdd.combineByKey[C](
+ createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine))
case None => None
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index 73d959331a..89c43ff935 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -17,10 +17,10 @@
package org.apache.spark.streaming.dstream
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD}
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.Duration
import scala.reflect.ClassTag
@@ -51,6 +51,14 @@ class WindowedDStream[T: ClassTag](
override def compute(validTime: Time): Option[RDD[T]] = {
val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
- Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
+ val rddsInWindow = parent.slice(currentWindow)
+ val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) {
+ logDebug("Using partition aware union for windowing at " + validTime)
+ new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow)
+ } else {
+ logDebug("Using normal union for windowing at " + validTime)
+ new UnionRDD(ssc.sc,rddsInWindow)
+ }
+ Some(windowRDD)
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index 1cd0b9b0a4..5f8be93a98 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -17,11 +17,18 @@
package org.apache.spark.streaming.scheduler
+import akka.actor.{Props, Actor}
import org.apache.spark.SparkEnv
import org.apache.spark.Logging
import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter}
import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock}
+/** Event classes for JobGenerator */
+private[scheduler] sealed trait JobGeneratorEvent
+private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent
+private[scheduler] case class ClearOldMetadata(time: Time) extends JobGeneratorEvent
+private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent
+
/**
* This class generates jobs from DStreams as well as drives checkpointing and cleaning
* up DStream metadata.
@@ -29,52 +36,75 @@ import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock}
private[streaming]
class JobGenerator(jobScheduler: JobScheduler) extends Logging {
- initLogging()
val ssc = jobScheduler.ssc
- val clockClass = System.getProperty(
- "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
- val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
- val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
- longTime => generateJobs(new Time(longTime)))
val graph = ssc.graph
+ val eventProcessorActor = ssc.env.actorSystem.actorOf(Props(new Actor {
+ def receive = {
+ case event: JobGeneratorEvent =>
+ logDebug("Got event of type " + event.getClass.getName)
+ processEvent(event)
+ }
+ }))
+ val clock = {
+ val clockClass = ssc.sc.conf.get(
+ "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
+ Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+ }
+ val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
+ longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))
lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
- new CheckpointWriter(ssc.checkpointDir)
+ new CheckpointWriter(ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
} else {
null
}
- var latestTime: Time = null
-
def start() = synchronized {
if (ssc.isCheckpointPresent) {
restart()
} else {
startFirstTime()
}
- logInfo("JobGenerator started")
}
-
- def stop() = synchronized {
+
+ def stop() {
timer.stop()
if (checkpointWriter != null) checkpointWriter.stop()
ssc.graph.stop()
logInfo("JobGenerator stopped")
}
+ /**
+ * On batch completion, clear old metadata and checkpoint computation.
+ */
+ private[scheduler] def onBatchCompletion(time: Time) {
+ eventProcessorActor ! ClearOldMetadata(time)
+ }
+
+ /** Processes all events */
+ private def processEvent(event: JobGeneratorEvent) {
+ event match {
+ case GenerateJobs(time) => generateJobs(time)
+ case ClearOldMetadata(time) => clearOldMetadata(time)
+ case DoCheckpoint(time) => doCheckpoint(time)
+ }
+ }
+
+ /** Starts the generator for the first time */
private def startFirstTime() {
val startTime = new Time(timer.getStartTime())
graph.start(startTime - graph.batchDuration)
timer.start(startTime.milliseconds)
- logInfo("JobGenerator's timer started at " + startTime)
+ logInfo("JobGenerator started at " + startTime)
}
+ /** Restarts the generator based on the information in checkpoint */
private def restart() {
// If manual clock is being used for testing, then
// either set the manual clock to the last checkpointed time,
// or if the property is defined set it to that time
if (clock.isInstanceOf[ManualClock]) {
val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds
- val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
+ val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong
clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
}
@@ -99,7 +129,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
// Restart the timer
timer.start(restartTime.milliseconds)
- logInfo("JobGenerator's timer restarted at " + restartTime)
+ logInfo("JobGenerator restarted at " + restartTime)
}
/** Generate jobs and perform checkpoint for the given `time`. */
@@ -107,16 +137,13 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
SparkEnv.set(ssc.env)
logInfo("\n-----------------------------------------------------\n")
jobScheduler.runJobs(time, graph.generateJobs(time))
- latestTime = time
- doCheckpoint(time)
+ eventProcessorActor ! DoCheckpoint(time)
}
- /**
- * On batch completion, clear old metadata and checkpoint computation.
- */
- private[streaming] def onBatchCompletion(time: Time) {
+ /** Clear DStream metadata for the given `time`. */
+ private def clearOldMetadata(time: Time) {
ssc.graph.clearOldMetadata(time)
- doCheckpoint(time)
+ eventProcessorActor ! DoCheckpoint(time)
}
/** Perform checkpoint for the give `time`. */
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
index 9511ccfbed..9304fc1a93 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -25,15 +25,13 @@ import org.apache.spark.streaming._
/**
* This class schedules jobs to be run on Spark. It uses the JobGenerator to generate
- * the jobs and runs them using a thread pool. Number of threads
+ * the jobs and runs them using a thread pool. Number of threads
*/
private[streaming]
class JobScheduler(val ssc: StreamingContext) extends Logging {
- initLogging()
-
val jobSets = new ConcurrentHashMap[Time, JobSet]
- val numConcurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
+ val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt
val executor = Executors.newFixedThreadPool(numConcurrentJobs)
val generator = new JobGenerator(this)
val listenerBus = new StreamingListenerBus()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index 4a3993e3e3..1559f7a9f7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -39,7 +39,6 @@ import org.apache.hadoop.conf.Configuration
private[streaming]
object MasterFailureTest extends Logging {
- initLogging()
@volatile var killed = false
@volatile var killCount = 0
@@ -331,7 +330,6 @@ class TestOutputStream[T: ClassTag](
*/
private[streaming]
class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging {
- initLogging()
override def run() {
try {
@@ -366,7 +364,6 @@ class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread
private[streaming]
class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
extends Thread with Logging {
- initLogging()
override def run() {
val localTestDir = Files.createTempDir()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index fc8655a083..6585d494a6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.util
import java.nio.ByteBuffer
import org.apache.spark.util.{RateLimitedOutputStream, IntParam}
import java.net.ServerSocket
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, Logging}
import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
import scala.io.Source
import java.io.IOException
@@ -42,7 +42,7 @@ object RawTextSender extends Logging {
// Repeat the input data multiple times to fill in a buffer
val lines = Source.fromFile(file).getLines().toArray
val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
- val ser = new KryoSerializer().newInstance()
+ val ser = new KryoSerializer(new SparkConf()).newInstance()
val serStream = ser.serializeStream(bufferStream)
var i = 0
while (bufferStream.position < blockSize) {
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index 62187957d6..0d2145da9a 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -19,6 +19,7 @@ package org.apache.spark.streaming;
import scala.Tuple2;
+import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import java.io.*;
@@ -28,6 +29,7 @@ import com.google.common.base.Optional;
import com.google.common.collect.Lists;
import com.google.common.io.Files;
+import org.apache.spark.SparkConf;
import org.apache.spark.HashPartitioner;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
@@ -67,7 +69,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa
Arrays.asList("hello", "world"),
Arrays.asList("goodnight", "moon"));
- List<List<Integer>> expected = Arrays.asList(
+ List<List<Integer>> expected = Arrays.asList(
Arrays.asList(5,5),
Arrays.asList(9,4));
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index b35ca00b53..ee6b433d1f 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -23,9 +23,9 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import util.ManualClock
+import org.apache.spark.{SparkContext, SparkConf}
class BasicOperationsSuite extends TestSuiteBase {
-
test("map") {
val input = Seq(1 to 4, 5 to 8, 9 to 12)
testOperation(
@@ -375,7 +375,11 @@ class BasicOperationsSuite extends TestSuiteBase {
}
test("slice") {
- val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
+ val conf2 = new SparkConf()
+ .setMaster("local[2]")
+ .setAppName("BasicOperationsSuite")
+ .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ val ssc = new StreamingContext(new SparkContext(conf2), Seconds(1))
val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
val stream = new TestInputStream[Int](ssc, input, 2)
ssc.registerInputStream(stream)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index 67a0841535..8dc80ac2ed 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -17,24 +17,18 @@
package org.apache.spark.streaming
-import dstream.FileInputDStream
-import org.apache.spark.streaming.StreamingContext._
import java.io.File
import scala.collection.mutable.ArrayBuffer
import scala.reflect.ClassTag
-
import org.apache.commons.io.FileUtils
-import org.scalatest.BeforeAndAfter
-
import com.google.common.io.Files
-
-import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
+import org.apache.hadoop.fs.{Path, FileSystem}
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.streaming.StreamingContext._
import org.apache.spark.streaming.dstream.FileInputDStream
import org.apache.spark.streaming.util.ManualClock
-
-
/**
* This test suites tests the checkpointing functionality of DStreams -
* the checkpointing of a DStream's RDDs as well as the checkpointing of
@@ -63,10 +57,10 @@ class CheckpointSuite extends TestSuiteBase {
assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
val stateStreamCheckpointInterval = Seconds(1)
-
+ val fs = FileSystem.getLocal(new Configuration())
// this ensure checkpointing occurs at least once
val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
val secondNumBatches = firstNumBatches
@@ -90,11 +84,12 @@ class CheckpointSuite extends TestSuiteBase {
ssc.start()
advanceTimeWithRealDelay(ssc, firstNumBatches)
logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData)
- assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure")
+ assert(!stateStream.checkpointData.checkpointFiles.isEmpty,
+ "No checkpointed RDDs in state stream before first failure")
stateStream.checkpointData.checkpointFiles.foreach {
- case (time, data) => {
- val file = new File(data.toString)
- assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist")
+ case (time, file) => {
+ assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time +
+ " for state stream before first failure does not exist")
}
}
@@ -102,7 +97,8 @@ class CheckpointSuite extends TestSuiteBase {
// and check whether the earlier checkpoint files are deleted
val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2))
advanceTimeWithRealDelay(ssc, secondNumBatches)
- checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
+ checkpointFiles.foreach(file =>
+ assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
ssc.stop()
// Restart stream computation using the checkpoint file and check whether
@@ -110,19 +106,20 @@ class CheckpointSuite extends TestSuiteBase {
ssc = new StreamingContext(checkpointDir)
stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
- assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure")
+ assert(!stateStream.generatedRDDs.isEmpty,
+ "No restored RDDs in state stream after recovery from first failure")
// Run one batch to generate a new checkpoint file and check whether some RDD
// is present in the checkpoint data or not
ssc.start()
advanceTimeWithRealDelay(ssc, 1)
- assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure")
+ assert(!stateStream.checkpointData.checkpointFiles.isEmpty,
+ "No checkpointed RDDs in state stream before second failure")
stateStream.checkpointData.checkpointFiles.foreach {
- case (time, data) => {
- val file = new File(data.toString)
- assert(file.exists(),
- "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist")
+ case (time, file) => {
+ assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time +
+ " for state stream before seconds failure does not exist")
}
}
ssc.stop()
@@ -132,10 +129,12 @@ class CheckpointSuite extends TestSuiteBase {
ssc = new StreamingContext(checkpointDir)
stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
- assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure")
+ assert(!stateStream.generatedRDDs.isEmpty,
+ "No restored RDDs in state stream after recovery from second failure")
- // Adjust manual clock time as if it is being restarted after a delay
- System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
+ // Adjust manual clock time as if it is being restarted after a delay; this is a hack because
+ // we modify the conf object, but it works for this one property
+ ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
ssc.start()
advanceTimeWithRealDelay(ssc, 4)
ssc.stop()
@@ -143,6 +142,7 @@ class CheckpointSuite extends TestSuiteBase {
ssc = null
}
+
// This tests whether the systm can recover from a master failure with simple
// non-stateful operations. This assumes as reliable, replayable input
// source - TestInputDStream.
@@ -191,15 +191,12 @@ class CheckpointSuite extends TestSuiteBase {
testCheckpointedOperation(input, operation, output, 7)
}
+
// This tests whether file input stream remembers what files were seen before
// the master failure and uses them again to process a large window operation.
// It also tests whether batches, whose processing was incomplete due to the
// failure, are re-processed or not.
test("recovery with file input stream") {
- // Disable manual clock as FileInputDStream does not work with manual clock
- val clockProperty = System.getProperty("spark.streaming.clock")
- System.clearProperty("spark.streaming.clock")
-
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
var ssc = new StreamingContext(master, framework, Seconds(1))
@@ -296,10 +293,6 @@ class CheckpointSuite extends TestSuiteBase {
)
// To ensure that all the inputs were received correctly
assert(expectedOutput.last === output.last)
-
- // Enable manual clock back again for other tests
- if (clockProperty != null)
- System.setProperty("spark.streaming.clock", clockProperty)
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 0cffed64a7..a8e053278a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -49,9 +49,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
testServer.start()
// Set up the streaming context and input streams
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
- val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]]
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
val outputStream = new TestOutputStream(networkStream, outputBuffer)
def output = outputBuffer.flatMap(x => x)
ssc.registerOutputStream(outputStream)
@@ -94,11 +94,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
test("file input stream") {
// Disable manual clock as FileInputDStream does not work with manual clock
- System.clearProperty("spark.streaming.clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val fileStream = ssc.textFileStream(testDir.toString)
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
def output = outputBuffer.flatMap(x => x)
@@ -139,7 +139,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
FileUtils.deleteDirectory(testDir)
// Enable manual clock back again for other tests
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
}
@@ -150,7 +150,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
testServer.start()
// Set up the streaming context and input streams
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor",
StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
@@ -193,6 +193,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
}
+
test("multi-thread receiver") {
// set up the test receiver
val numThreads = 10
@@ -202,7 +203,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
MultiThreadTestReceiver.haveAllThreadsFinished = false
// set up the network stream using the test receiver
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.networkStream[Int](testReceiver)
val countStream = networkStream.count
val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]]
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index f56c0462f4..b20d02f996 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -28,7 +28,7 @@ import java.io.{ObjectInputStream, IOException}
import org.scalatest.{BeforeAndAfter, FunSuite}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, SparkConf, Logging}
import org.apache.spark.rdd.RDD
/**
@@ -133,19 +133,22 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Whether to actually wait in real time before changing manual clock
def actuallyWait = false
+ //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
+ val conf = new SparkConf()
+ .setMaster(master)
+ .setAppName(framework)
+ .set("spark.cleaner.ttl", "3600")
+
// Default before function for any streaming test suite. Override this
// if you want to add your stuff to "before" (i.e., don't call before { } )
def beforeFunction() {
if (useManualClock) {
logInfo("Using manual clock")
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
} else {
logInfo("Using real clock")
- System.clearProperty("spark.streaming.clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
}
- // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
- System.clearProperty("spark.driver.port")
- System.clearProperty("spark.hostPort")
}
// Default after function for any streaming test suite. Override this
@@ -168,9 +171,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
operation: DStream[U] => DStream[V],
numPartitions: Int = numInputPartitions
): StreamingContext = {
-
// Create StreamingContext
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
@@ -194,9 +196,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W]
): StreamingContext = {
-
// Create StreamingContext
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
index 6b4aaefcdf..c39abfc21b 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
@@ -18,7 +18,6 @@
package org.apache.spark.streaming
import org.apache.spark.streaming.StreamingContext._
-import collection.mutable.ArrayBuffer
class WindowOperationsSuite extends TestSuiteBase {
@@ -225,9 +224,7 @@ class WindowOperationsSuite extends TestSuiteBase {
val slideDuration = Seconds(1)
val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
val operation = (s: DStream[(String, Int)]) => {
- s.groupByKeyAndWindow(windowDuration, slideDuration)
- .map(x => (x._1, x._2.toSet))
- .persist()
+ s.groupByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toSet))
}
testOperation(input, operation, expectedOutput, numBatches, true)
}