diff options
Diffstat (limited to 'streaming/src')
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala (renamed from streaming/src/main/scala/spark/streaming/Checkpoint.scala) | 28 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/DStream.scala (renamed from streaming/src/main/scala/spark/streaming/DStream.scala) | 17 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala (renamed from streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala) | 4 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala (renamed from streaming/src/main/scala/spark/streaming/DStreamGraph.scala) | 4 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Duration.scala (renamed from streaming/src/main/scala/spark/streaming/Duration.scala) | 10 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Interval.scala (renamed from streaming/src/main/scala/spark/streaming/Interval.scala) | 2 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Job.scala (renamed from streaming/src/main/scala/spark/streaming/Job.scala) | 2 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala (renamed from streaming/src/main/scala/spark/streaming/JobManager.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala (renamed from streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala) | 12 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala (renamed from streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala) | 29 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala (renamed from streaming/src/main/scala/spark/streaming/Scheduler.scala) | 9 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala (renamed from streaming/src/main/scala/spark/streaming/StreamingContext.scala) | 21 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/Time.scala (renamed from streaming/src/main/scala/spark/streaming/Time.scala) | 2 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala (renamed from streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala) | 16 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala (renamed from streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala) | 10 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala (renamed from streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala) | 30 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala (renamed from streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala) | 34 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala) | 9 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala) | 17 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala) | 4 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala) | 29 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala) | 4 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala) | 16 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala) | 9 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala) | 12 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala (renamed from streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala) | 10 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala (renamed from streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala) | 8 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala (renamed from streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala) | 4 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala (renamed from streaming/src/main/scala/spark/streaming/util/Clock.scala) | 2 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala (renamed from streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala) | 9 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala (renamed from streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala) | 6 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala (renamed from streaming/src/main/scala/spark/streaming/util/RawTextSender.scala) | 7 | ||||
-rw-r--r-- | streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala (renamed from streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala) | 2 | ||||
-rw-r--r-- | streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java (renamed from streaming/src/test/java/spark/streaming/JavaAPISuite.java) | 32 | ||||
-rw-r--r-- | streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala (renamed from streaming/src/test/java/spark/streaming/JavaTestUtils.scala) | 17 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala (renamed from streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala) | 6 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala (renamed from streaming/src/test/scala/spark/streaming/CheckpointSuite.scala) | 8 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala (renamed from streaming/src/test/scala/spark/streaming/FailureSuite.scala) | 6 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala (renamed from streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala) | 12 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala (renamed from streaming/src/test/scala/spark/streaming/TestSuiteBase.scala) | 11 | ||||
-rw-r--r-- | streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala (renamed from streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala) | 6 |
58 files changed, 312 insertions, 288 deletions
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 1e4c1e3742..2d8f072624 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -15,18 +15,19 @@ * limitations under the License. */ -package spark.streaming - -import spark.{Logging, Utils} - -import org.apache.hadoop.fs.{FileUtil, Path} -import org.apache.hadoop.conf.Configuration +package org.apache.spark.streaming import java.io._ -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import java.util.concurrent.Executors import java.util.concurrent.RejectedExecutionException +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.Logging +import org.apache.spark.io.CompressionCodec + + private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { @@ -49,6 +50,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } + /** * Convenience class to speed up the writing of graph checkpoint to file */ @@ -66,6 +68,8 @@ class CheckpointWriter(checkpointDir: String) extends Logging { val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) + private val compressionCodec = CompressionCodec.createCodec() + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since // I did not notice any errors - reintroduce it ? @@ -103,7 +107,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { def write(checkpoint: Checkpoint) { val bos = new ByteArrayOutputStream() - val zos = new LZFOutputStream(bos) + val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) oos.writeObject(checkpoint) oos.close() @@ -137,6 +141,8 @@ object CheckpointReader extends Logging { 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 compressionCodec = CompressionCodec.createCodec() + attempts.foreach(file => { if (fs.exists(file)) { logInfo("Attempting to load checkpoint from file '" + file + "'") @@ -147,7 +153,7 @@ object CheckpointReader extends Logging { // of ObjectInputStream is used to explicitly use the current thread's default class // 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 = new LZFInputStream(fis) + val zis = compressionCodec.compressedInputStream(fis) val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader) val cp = ois.readObject.asInstanceOf[Checkpoint] ois.close() @@ -170,7 +176,9 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) + extends ObjectInputStream(inputStream_) { + override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { return loader.loadClass(desc.getName()) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 684d3abb56..80da6bd30b 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -15,14 +15,17 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream._ +import org.apache.spark.streaming.dstream._ import StreamingContext._ +import org.apache.spark.util.MetadataCleaner + //import Time._ -import spark.{RDD, Logging} -import spark.storage.StorageLevel +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -34,7 +37,7 @@ import org.apache.hadoop.conf.Configuration /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous - * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]] + * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]] * for more details on RDDs). DStreams can either be created from live data (such as, data from * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each @@ -42,7 +45,7 @@ import org.apache.hadoop.conf.Configuration * by a parent DStream. * * This class contains the basic operations available on all DStreams, such as `map`, `filter` and - * `window`. In addition, [[spark.streaming.PairDStreamFunctions]] contains operations available + * `window`. In addition, [[org.apache.spark.streaming.PairDStreamFunctions]] contains operations available * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through * implicit conversions when `spark.streaming.StreamingContext._` is imported. @@ -209,7 +212,7 @@ abstract class DStream[T: ClassManifest] ( checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." ) - val metadataCleanerDelay = spark.util.MetadataCleaner.getDelaySeconds + val metadataCleanerDelay = MetadataCleaner.getDelaySeconds logInfo("metadataCleanupDelay = " + metadataCleanerDelay) assert( metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala index 399ca1c63d..58a0da2870 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.conf.Configuration import collection.mutable.HashMap -import spark.Logging +import org.apache.spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index c09a332d44..b9a58fded6 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import dstream.InputDStream import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import collection.mutable.ArrayBuffer -import spark.Logging +import org.apache.spark.Logging final private[streaming] class DStreamGraph extends Serializable with Logging { initLogging() diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 12a14e233d..6bf275f5af 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Utils +import org.apache.spark.util.Utils case class Duration (private val millis: Long) { @@ -57,7 +57,7 @@ case class Duration (private val millis: Long) { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of milliseconds. */ object Milliseconds { @@ -65,7 +65,7 @@ object Milliseconds { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of seconds. */ object Seconds { @@ -73,7 +73,7 @@ object Seconds { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of minutes. */ object Minutes { diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index b30cd969e9..04c994c136 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming private[streaming] class Interval(val beginTime: Time, val endTime: Time) { diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala index ceb3f92b65..2128b7c7a6 100644 --- a/streaming/src/main/scala/spark/streaming/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import java.util.concurrent.atomic.AtomicLong diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala index a31230689f..5233129506 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Logging -import spark.SparkEnv +import org.apache.spark.Logging +import org.apache.spark.SparkEnv import java.util.concurrent.Executors import collection.mutable.HashMap import collection.mutable.ArrayBuffer diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala index d4cf2e568c..aae79a4e6f 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} -import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} -import spark.Logging -import spark.SparkEnv -import spark.SparkContext._ +import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} +import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} +import org.apache.spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.Queue diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala index 47bf07bee1..757bc98981 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ -import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} -import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} -import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} +import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} +import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import spark.{Manifests, RDD, Partitioner, HashPartitioner} -import spark.SparkContext._ -import spark.storage.StorageLevel +import org.apache.spark.{Partitioner, HashPartitioner} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{Manifests, RDD, PairRDDFunctions} +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer @@ -60,7 +61,7 @@ extends Serializable { } /** - * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]] + * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[org.apache.spark.Partitioner]] * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { @@ -91,7 +92,7 @@ extends Serializable { /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the + * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the * partitioning of each RDD. */ def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = { @@ -101,8 +102,8 @@ extends Serializable { /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more - * information. + * combineByKey for RDDs. Please refer to combineByKey in + * [[org.apache.spark.rdd.PairRDDFunctions]] for more information. */ def combineByKey[C: ClassManifest]( createCombiner: V => C, @@ -360,7 +361,7 @@ extends Serializable { /** * Create a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[spark.Partitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. @@ -379,7 +380,7 @@ extends Serializable { /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. Note, that * this function may generate a different a tuple with a different key diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index 252cc2a303..ed892e33e6 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import util.{ManualClock, RecurringTimer, Clock} -import spark.SparkEnv -import spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.Logging private[streaming] class Scheduler(ssc: StreamingContext) extends Logging { @@ -34,7 +34,8 @@ class Scheduler(ssc: StreamingContext) extends Logging { null } - val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock") + 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))) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ffd656227d..878725c705 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -15,21 +15,22 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe -import spark.streaming.dstream._ +import org.apache.spark.streaming.dstream._ -import spark._ -import spark.streaming.receivers.ActorReceiver -import spark.streaming.receivers.ReceiverSupervisorStrategy -import spark.streaming.receivers.ZeroMQReceiver -import spark.storage.StorageLevel -import spark.util.MetadataCleaner -import spark.streaming.receivers.ActorReceiver +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.receivers.ActorReceiver +import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy +import org.apache.spark.streaming.receivers.ZeroMQReceiver +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.MetadataCleaner +import org.apache.spark.streaming.receivers.ActorReceiver import scala.collection.mutable.Queue import scala.collection.Map @@ -183,6 +184,7 @@ class StreamingContext private ( /** * Create an input stream with any arbitrary user implemented network receiver. + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of NetworkReceiver */ def networkStream[T: ClassManifest]( @@ -195,6 +197,7 @@ class StreamingContext private ( /** * Create an input stream with any arbitrary user implemented actor receiver. + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor * @param storageLevel RDD storage level. Defaults to memory-only. diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index ad5eab9dd2..2678334f53 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming /** * This is a simple class that represents an absolute instant of time. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 7dcb1d713d..d1932b6b05 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java -import spark.streaming.{Duration, Time, DStream} -import spark.api.java.function.{Function => JFunction} -import spark.api.java.JavaRDD -import spark.storage.StorageLevel -import spark.RDD +import org.apache.spark.streaming.{Duration, Time, DStream} +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.rdd.RDD /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous - * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]] + * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]] * for more details on RDDs). DStreams can either be created from live data (such as, data from * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each @@ -33,7 +33,7 @@ import spark.RDD * by a parent DStream. * * This class contains the basic operations available on all DStreams, such as `map`, `filter` and - * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available + * `window`. In addition, [[org.apache.spark.streaming.api.java.JavaPairDStream]] contains operations available * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. * * DStreams internally is characterized by a few basic properties: diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 3ab5c1fdde..459695b7ca 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ -import spark.streaming._ -import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} -import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} +import org.apache.spark.streaming._ +import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util -import spark.RDD +import org.apache.spark.rdd.RDD import JavaDStream._ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ccd15563b0..978fca33ad 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -15,24 +15,25 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ -import spark.streaming._ -import spark.streaming.StreamingContext._ -import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import spark.{RDD, Partitioner} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.Partitioner import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration -import spark.api.java.{JavaRDD, JavaPairRDD} -import spark.storage.StorageLevel +import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} +import org.apache.spark.storage.StorageLevel import com.google.common.base.Optional -import spark.RDD +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.PairRDDFunctions class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], @@ -114,7 +115,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. * Therefore, the values for each key in `this` DStream's RDDs are grouped into a - * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]] + * single sequence to generate the RDDs of the new DStream. [[org.apache.spark.Partitioner]] * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = @@ -138,7 +139,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the + * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the * partitioning of each RDD. */ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = { @@ -147,7 +148,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 [[spark.PairRDDFunctions]] for more + * combineByKey for RDDs. Please refer to combineByKey in [[PairRDDFunctions]] for more * information. */ def combineByKey[C](createCombiner: JFunction[V, C], @@ -401,10 +402,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( (Seq[V], Option[S]) => Option[S] = { val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => { val list: JList[V] = values - val scalaState: Optional[S] = state match { - case Some(s) => Optional.of(s) - case _ => Optional.absent() - } + val scalaState: Optional[S] = JavaUtils.optionToOptional(state) val result: Optional[S] = in.apply(list, scalaState) result.isPresent match { case true => Some(result.get()) @@ -448,7 +446,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Create a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[spark.Partitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b7720ad0ea..54ba3e6025 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -15,25 +15,29 @@ * limitations under the License. */ -package spark.streaming.api.java - -import spark.streaming._ -import receivers.{ActorReceiver, ReceiverSupervisorStrategy} -import spark.streaming.dstream._ -import spark.storage.StorageLevel -import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import spark.api.java.{JavaSparkContext, JavaRDD} +package org.apache.spark.streaming.api.java + +import java.lang.{Long => JLong, Integer => JInt} +import java.io.InputStream +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ + import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe -import scala.collection.JavaConversions._ -import java.lang.{Long => JLong, Integer => JInt} -import java.io.InputStream -import java.util.{Map => JMap} import twitter4j.auth.Authorization +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.{JavaSparkContext, JavaRDD} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy} + /** * 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 @@ -537,7 +541,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue) } @@ -554,7 +558,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime) } @@ -575,7 +579,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { defaultRDD: JavaRDD[T]): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 99553d295d..4eddc755b9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.{RDD, Partitioner} -import spark.rdd.CoGroupedRDD -import spark.streaming.{Time, DStream, Duration} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.CoGroupedRDD +import org.apache.spark.streaming.{Time, DStream, Duration} private[streaming] class CoGroupedDStream[K : ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 095137092a..a9a05c9981 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Time, StreamingContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Time, StreamingContext} /** * An input stream that always returns the same RDD on each timestep. Useful for testing. diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index de0536125d..fea0573b77 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD -import spark.streaming.{DStreamCheckpointData, StreamingContext, Time} +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time} import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.conf.Configuration diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index 9d8c5c3175..91ee2c1a36 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD private[streaming] class FilteredDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 78d7117f0f..ca7d7ca49e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD -import spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ private[streaming] class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index d13bebb10f..b37966f9a7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD private[streaming] class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala index 4906f503c2..18de772946 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext +import java.net.InetSocketAddress +import java.io.{ObjectInput, ObjectOutput, Externalizable} +import java.nio.ByteBuffer -import spark.Utils -import spark.storage.StorageLevel +import scala.collection.JavaConversions._ import org.apache.flume.source.avro.AvroSourceProtocol import org.apache.flume.source.avro.AvroFlumeEvent @@ -28,11 +29,9 @@ import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer -import scala.collection.JavaConversions._ - -import java.net.InetSocketAddress -import java.io.{ObjectInput, ObjectOutput, Externalizable} -import java.nio.ByteBuffer +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.util.Utils +import org.apache.spark.storage.StorageLevel private[streaming] class FlumeInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 7df537eb56..e21bac4602 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Duration, DStream, Job, Time} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, DStream, Job, Time} private[streaming] class ForEachDStream[T: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index 06fda6fe8e..4294b07d91 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD private[streaming] class GlommedDStream[T: ClassManifest](parent: DStream[T]) diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 4dbdec459d..674b27118c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Time, Duration, StreamingContext, DStream} +import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream} /** * This is the abstract base class for all input streams. This class provides to methods diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala index 6ee588af15..51e913675d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.Logging -import spark.storage.StorageLevel -import spark.streaming.{Time, DStreamCheckpointData, StreamingContext} +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Time, DStreamCheckpointData, StreamingContext} import java.util.Properties import java.util.concurrent.Executors diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index af41a1b9ac..5329601a6f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD private[streaming] class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 8d8a6161c6..8290df90a2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD -import spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ private[streaming] class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index 3fda84a38a..b1682afea3 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD private[streaming] class MappedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 344b41c4d0..31f9891560 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -15,30 +15,29 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} - -import spark.{Logging, SparkEnv, RDD} -import spark.rdd.BlockRDD -import spark.storage.StorageLevel +import java.util.concurrent.ArrayBlockingQueue +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import java.nio.ByteBuffer - import akka.actor.{Props, Actor} import akka.pattern.ask import akka.dispatch.Await import akka.util.duration._ -import spark.streaming.util.{RecurringTimer, SystemClock} -import java.util.concurrent.ArrayBlockingQueue + +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.streaming._ +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.storage.StorageLevel /** * Abstract class for defining any InputDStream that has to start a receiver on worker * nodes to receive external data. Specific implementations of NetworkInputDStream must * define the getReceiver() function that gets the receiver object of type - * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive + * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive * data. * @param ssc_ Streaming context that will execute this input stream * @tparam T Class type of the object of this stream @@ -83,7 +82,7 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe /** * Abstract class of a receiver that can be run on worker nodes to receive external data. See - * [[spark.streaming.dstream.NetworkInputDStream]] for an explanation. + * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation. */ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging { @@ -145,8 +144,8 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log } /** - * Stops the receiver and reports to exception to the tracker. - * This should be called whenever an exception has happened on any thread + * Stops the receiver and reports exception to the tracker. + * This should be called whenever an exception is to be handled on any thread * of the receiver. */ protected def stopOnError(e: Exception) { @@ -202,7 +201,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log } /** - * Batches objects created by a [[spark.streaming.NetworkReceiver]] and puts them into + * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts them into * appropriately named blocks at regular intervals. This class starts two threads, * one to periodically start a new batch and prepare the previous batch of as a block, * the other to push the blocks into the block manager. diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 33f7cd063f..15782f5c11 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext private[streaming] class PluggableInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index b269061b73..7d9f3521b1 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.UnionRDD import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer -import spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{Time, StreamingContext} private[streaming] class QueueInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index 236f74f575..c91f12ecd7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.Logging -import spark.storage.StorageLevel -import spark.streaming.StreamingContext +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext import java.net.InetSocketAddress import java.nio.ByteBuffer diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 96260501ab..b88a4db959 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ -import spark.RDD -import spark.rdd.{CoGroupedRDD, MapPartitionsRDD} -import spark.Partitioner -import spark.SparkContext._ -import spark.storage.StorageLevel +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} +import org.apache.spark.Partitioner +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer -import spark.streaming.{Duration, Interval, Time, DStream} +import org.apache.spark.streaming.{Duration, Interval, Time, DStream} private[streaming] class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 83b57b27f7..a95e66d761 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.{RDD, Partitioner} -import spark.SparkContext._ -import spark.streaming.{Duration, DStream, Time} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 5877b10e0e..e2539c7396 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext -import spark.storage.StorageLevel -import spark.util.NextIterator +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.NextIterator import java.io._ import java.net.Socket diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 4b46613d5e..362a6bf4cc 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.Partitioner -import spark.SparkContext._ -import spark.storage.StorageLevel -import spark.streaming.{Duration, Time, DStream} +import org.apache.spark.rdd.RDD +import org.apache.spark.Partitioner +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, Time, DStream} private[streaming] class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index e7fbc5bbcf..60485adef9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] class TransformedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala index f09a8b9f90..387e15b0e6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark._ -import spark.streaming._ +import org.apache.spark._ +import org.apache.spark.streaming._ import storage.StorageLevel import twitter4j._ import twitter4j.auth.Authorization diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index 3eaa9a7e7f..c696bb70a8 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.rdd.RDD import collection.mutable.ArrayBuffer -import spark.rdd.UnionRDD +import org.apache.spark.rdd.UnionRDD private[streaming] class UnionDStream[T: ClassManifest](parents: Array[DStream[T]]) diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index fd24d61730..3c57294269 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD -import spark.storage.StorageLevel -import spark.streaming.{Duration, Interval, Time, DStream} +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, Interval, Time, DStream} private[streaming] class WindowedDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index 2d9937eab8..4b5d8c467e 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming.receivers +package org.apache.spark.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } import akka.actor.{ actorRef2Scala, ActorRef } import akka.actor.{ PossiblyHarmful, OneForOneStrategy } -import spark.storage.StorageLevel -import spark.streaming.dstream.NetworkReceiver +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger @@ -45,6 +45,8 @@ object ReceiverSupervisorStrategy { * A receiver trait to be mixed in with your Actor to gain access to * pushBlock API. * + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html + * * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala index 22d554e7e4..043bb8c8bf 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.receivers +package org.apache.spark.streaming.receivers import akka.actor.Actor import akka.zeromq._ -import spark.Logging +import org.apache.spark.Logging /** * A receiver to subscribe to ZeroMQ stream. diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index d9ac722df5..f67bb2f6ac 100644 --- a/streaming/src/main/scala/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util private[streaming] trait Clock { diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 8ce5d8daf5..6977957126 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -15,11 +15,12 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util -import spark.{Logging, RDD} -import spark.streaming._ -import spark.streaming.dstream.ForEachDStream +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream.ForEachDStream import StreamingContext._ import scala.util.Random diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bf04120293..4e6ce6eabd 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import scala.collection.JavaConversions.mapAsScalaMap diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 5cc6ad9dee..fc8655a083 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -15,15 +15,16 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util import java.nio.ByteBuffer -import spark.util.{RateLimitedOutputStream, IntParam} +import org.apache.spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket -import spark.{Logging, KryoSerializer} +import org.apache.spark.{Logging} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException +import org.apache.spark.serializer.KryoSerializer /** * A helper program that sends blocks of Kryo-serialized text strings out on a socket at a diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 7ecc44236d..d644240405 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util private[streaming] class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) { diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 3b93790baa..c0d729ff87 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming; +package org.apache.spark.streaming; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -28,20 +28,20 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; import scala.Tuple2; -import spark.HashPartitioner; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaRDDLike; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.*; -import spark.storage.StorageLevel; -import spark.streaming.api.java.JavaDStream; -import spark.streaming.api.java.JavaPairDStream; -import spark.streaming.api.java.JavaStreamingContext; -import spark.streaming.JavaTestUtils; -import spark.streaming.JavaCheckpointTestUtils; -import spark.streaming.InputStreamsSuite; +import org.apache.spark.HashPartitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaRDDLike; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.JavaTestUtils; +import org.apache.spark.streaming.JavaCheckpointTestUtils; +import org.apache.spark.streaming.InputStreamsSuite; import java.io.*; import java.util.*; @@ -59,7 +59,7 @@ public class JavaAPISuite implements Serializable { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock"); + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index f9d25db8da..8a6604904d 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -15,20 +15,21 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import java.util.{List => JList} -import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} -import spark.streaming._ +import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming._ import java.util.ArrayList import collection.JavaConversions._ +import org.apache.spark.api.java.JavaRDDLike /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { /** - * Create a [[spark.streaming.TestInputStream]] and attach it to the supplied context. + * Create a [[org.apache.spark.streaming.TestInputStream]] and attach it to the supplied context. * The stream will be derived from the supplied lists of Java objects. **/ def attachTestInputStream[T]( @@ -46,11 +47,11 @@ trait JavaTestBase extends TestSuiteBase { /** * Attach a provided stream to it's associated StreamingContext as a - * [[spark.streaming.TestOutputStream]]. + * [[org.apache.spark.streaming.TestOutputStream]]. **/ - def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R], - R <: spark.api.java.JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = { + def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( + dstream: JavaDStreamLike[T, This, R]) = + { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] val ostream = new TestOutputStream(dstream.dstream, diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 67e3e0cd30..11586f72b6 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import scala.runtime.RichInt import util.ManualClock @@ -26,7 +26,7 @@ class BasicOperationsSuite extends TestSuiteBase { override def framework() = "BasicOperationsSuite" before { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 8c639648f0..a327de80b3 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import dstream.FileInputDStream -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import java.io.File import runtime.RichInt import org.scalatest.BeforeAndAfter @@ -36,7 +36,7 @@ import com.google.common.io.Files */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") before { FileUtils.deleteDirectory(new File(checkpointDir)) @@ -63,7 +63,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 7fc649fe27..6337c5359c 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Logging -import spark.streaming.util.MasterFailureTest +import org.apache.spark.Logging +import org.apache.spark.streaming.util.MasterFailureTest import StreamingContext._ import org.scalatest.{FunSuite, BeforeAndAfter} diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 1c5419b16d..42e3e51e3f 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import akka.actor.Actor import akka.actor.IO @@ -29,9 +29,9 @@ import java.io.{File, BufferedWriter, OutputStreamWriter} import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.ManualClock -import spark.storage.StorageLevel -import spark.streaming.receivers.Receiver -import spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receivers.Receiver +import org.apache.spark.Logging import scala.util.Random import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter @@ -52,7 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" before { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { @@ -207,7 +207,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(testDir) // Enable manual clock back again for other tests - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index cb34b5a7cc..37dd9c4cc6 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream.{InputDStream, ForEachDStream} -import spark.streaming.util.ManualClock - -import spark.{RDD, Logging} +import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream} +import org.apache.spark.streaming.util.ManualClock import collection.mutable.ArrayBuffer import collection.mutable.SynchronizedBuffer @@ -29,6 +27,9 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD + /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index 894b765fc6..f50e05c0d8 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import collection.mutable.ArrayBuffer class WindowOperationsSuite extends TestSuiteBase { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") override def framework = "WindowOperationsSuite" |