You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/01/08 07:22:03 UTC

[01/13] git commit: Refactored streaming project to separate out the twitter functionality.

Updated Branches:
  refs/heads/master f5f12dc28 -> c0f0155ec


Refactored streaming project to separate out the twitter functionality.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/6e430396
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/6e430396
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/6e430396

Branch: refs/heads/master
Commit: 6e43039614ed1ec55a134fb82fb3e8d4e80996ef
Parents: e240bad
Author: Tathagata Das <ta...@gmail.com>
Authored: Thu Dec 26 18:02:49 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Thu Dec 26 18:02:49 2013 -0800

----------------------------------------------------------------------
 .../streaming/examples/TwitterAlgebirdCMS.scala |   2 +
 .../streaming/examples/TwitterAlgebirdHLL.scala |   2 +-
 .../streaming/examples/TwitterPopularTags.scala |   1 +
 .../twitter/StreamingContextWithTwitter.scala   |  27 +++++
 .../streaming/twitter/TwitterInputDStream.scala | 101 +++++++++++++++++++
 .../spark/streaming/twitter/package.scala       |   7 ++
 project/SparkBuild.scala                        |  13 ++-
 .../spark/streaming/StreamingContext.scala      |   6 +-
 .../api/java/JavaStreamingContext.scala         |   8 +-
 .../streaming/dstream/TwitterInputDStream.scala |  99 ------------------
 10 files changed, 158 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 35b6329..a00b3bd 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -23,6 +23,8 @@ import com.twitter.algebird._
 import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.SparkContext._
 
+import org.apache.spark.streaming.twitter._
+
 /**
  * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute
  * windowed and global Top-K estimates of user IDs occurring in a Twitter stream.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index 8bfde2a..8215606 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext}
 import org.apache.spark.storage.StorageLevel
 import com.twitter.algebird.HyperLogLog._
 import com.twitter.algebird.HyperLogLogMonoid
-import org.apache.spark.streaming.dstream.TwitterInputDStream
+import org.apache.spark.streaming.twitter._
 
 /**
  * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 27aa6b1..c4ded5e 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
 import org.apache.spark.streaming.{Seconds, StreamingContext}
 import StreamingContext._
 import org.apache.spark.SparkContext._
+import org.apache.spark.streaming.twitter._
 
 /**
  * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
new file mode 100644
index 0000000..fe66e28
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
@@ -0,0 +1,27 @@
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+
+class StreamingContextWithTwitter(ssc: StreamingContext) {
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
+   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
+   *        .consumerSecret, .accessToken and .accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      twitterAuth: Option[Authorization] = None,
+      filters: Seq[String] = Nil,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[Status] = {
+    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
new file mode 100644
index 0000000..97e48eb
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import java.util.prefs.Preferences
+import twitter4j._
+import twitter4j.auth.Authorization
+import twitter4j.conf.ConfigurationBuilder
+import twitter4j.conf.PropertyConfiguration
+import twitter4j.auth.OAuthAuthorization
+import twitter4j.auth.AccessToken
+import org.apache.spark._
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.storage.StorageLevel
+
+/* A stream of Twitter statuses, potentially filtered by one or more keywords.
+*
+* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials.
+* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is
+* such that this may return a sampled subset of all tweets during each interval.
+* 
+* If no Authorization object is provided, initializes OAuth authorization using the system
+* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret.
+*/
+private[streaming]
+class TwitterInputDStream(
+    @transient ssc_ : StreamingContext,
+    twitterAuth: Option[Authorization],
+    filters: Seq[String],
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[Status](ssc_)  {
+  
+  private def createOAuthAuthorization(): Authorization = {
+    new OAuthAuthorization(new ConfigurationBuilder().build())
+  }
+
+  private val authorization = twitterAuth.getOrElse(createOAuthAuthorization())
+  
+  override def getReceiver(): NetworkReceiver[Status] = {
+    new TwitterReceiver(authorization, filters, storageLevel)
+  }
+}
+
+private[streaming]
+class TwitterReceiver(
+    twitterAuth: Authorization,
+    filters: Seq[String],
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[Status] {
+
+  var twitterStream: TwitterStream = _
+  lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+  protected override def onStart() {
+    blockGenerator.start()
+    twitterStream = new TwitterStreamFactory().getInstance(twitterAuth)
+    twitterStream.addListener(new StatusListener {
+      def onStatus(status: Status) = {
+        blockGenerator += status
+      }
+      // Unimplemented
+      def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {}
+      def onTrackLimitationNotice(i: Int) {}
+      def onScrubGeo(l: Long, l1: Long) {}
+      def onStallWarning(stallWarning: StallWarning) {}
+      def onException(e: Exception) { stopOnError(e) }
+    })
+
+    val query: FilterQuery = new FilterQuery
+    if (filters.size > 0) {
+      query.track(filters.toArray)
+      twitterStream.filter(query)
+    } else {
+      twitterStream.sample()
+    }
+    logInfo("Twitter receiver started")
+  }
+
+  protected override def onStop() {
+    blockGenerator.stop()
+    twitterStream.shutdown()
+    logInfo("Twitter receiver stopped")
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
new file mode 100644
index 0000000..89c202a
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
@@ -0,0 +1,7 @@
+package org.apache.spark.streaming
+
+package object twitter {
+  implicit def enrichMyStreamingContext(ssc: StreamingContext): StreamingContextWithTwitter = {
+    new StreamingContextWithTwitter(ssc)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 7bcbd90..f9ff781 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -50,7 +50,7 @@ object SparkBuild extends Build {
     .dependsOn(core, bagel, mllib)
 
   lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
-    .dependsOn(core, mllib, bagel, streaming)
+    .dependsOn(core, mllib, bagel, streaming, externalTwitter)
 
   lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
 
@@ -60,6 +60,8 @@ object SparkBuild extends Build {
 
   lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
 
+  lazy val externalTwitter = Project("streaming-twitter", file("external/twitter"), settings = twitterSettings) dependsOn(streaming)
+
   lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
     .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
 
@@ -313,7 +315,7 @@ object SparkBuild extends Build {
         excludeAll(excludeNetty),
       "org.eclipse.paho"        % "mqtt-client"      % "0.4.0",
       "com.github.sgroschupf"   % "zkclient"         % "0.1"                   excludeAll(excludeNetty),
-      "org.twitter4j"           % "twitter4j-stream" % "3.0.3"                 excludeAll(excludeNetty),
+      // "org.twitter4j"           % "twitter4j-stream" % "3.0.3"                 excludeAll(excludeNetty),
       "org.spark-project.akka" %% "akka-zeromq"      % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
     )
   )
@@ -354,4 +356,11 @@ object SparkBuild extends Build {
       case _ => MergeStrategy.first
     }
   )
+
+  def twitterSettings() = streamingSettings ++ Seq(
+    name := "spark-twitter",
+    libraryDependencies ++= Seq(
+      "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty)
+    )
+  )
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 41da028..25b9b70 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -45,8 +45,8 @@ import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
 import org.apache.hadoop.fs.Path
-import twitter4j.Status
-import twitter4j.auth.Authorization
+//import twitter4j.Status
+//import twitter4j.auth.Authorization
 import org.apache.spark.streaming.scheduler._
 import akka.util.ByteString
 
@@ -414,6 +414,7 @@ class StreamingContext private (
     fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
   }
 
+  /*
   /**
    * Create a input stream that returns tweets received from Twitter.
    * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
@@ -431,6 +432,7 @@ class StreamingContext private (
     registerInputStream(inputStream)
     inputStream
   }
+  */
 
   /**
    * Create an input stream from a queue of RDDs. In each batch,

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
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 78d318c..b32cfbb 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
@@ -25,13 +25,13 @@ import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
 
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import twitter4j.Status
+//import twitter4j.Status
 import akka.actor.Props
 import akka.actor.SupervisorStrategy
 import akka.zeromq.Subscribe
 import akka.util.ByteString
 
-import twitter4j.auth.Authorization
+//import twitter4j.auth.Authorization
 
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
@@ -338,7 +338,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
     ssc.flumeStream(hostname, port)
   }
-
+  /*
   /**
    * Create a input stream that returns tweets received from Twitter.
    * @param twitterAuth Twitter4J Authorization object
@@ -409,7 +409,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def twitterStream(): JavaDStream[Status] = {
     ssc.twitterStream()
   }
-
+  */
   /**
    * Create an input stream with any arbitrary user implemented actor receiver.
    * @param props Props object defining creation of the actor

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6e430396/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
deleted file mode 100644
index 387e15b..0000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.dstream
-
-import org.apache.spark._
-import org.apache.spark.streaming._
-import storage.StorageLevel
-import twitter4j._
-import twitter4j.auth.Authorization
-import java.util.prefs.Preferences
-import twitter4j.conf.ConfigurationBuilder
-import twitter4j.conf.PropertyConfiguration
-import twitter4j.auth.OAuthAuthorization
-import twitter4j.auth.AccessToken
-
-/* A stream of Twitter statuses, potentially filtered by one or more keywords.
-*
-* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials.
-* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is
-* such that this may return a sampled subset of all tweets during each interval.
-* 
-* If no Authorization object is provided, initializes OAuth authorization using the system
-* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret.
-*/
-private[streaming]
-class TwitterInputDStream(
-    @transient ssc_ : StreamingContext,
-    twitterAuth: Option[Authorization],
-    filters: Seq[String],
-    storageLevel: StorageLevel
-  ) extends NetworkInputDStream[Status](ssc_)  {
-  
-  private def createOAuthAuthorization(): Authorization = {
-    new OAuthAuthorization(new ConfigurationBuilder().build())
-  }
-
-  private val authorization = twitterAuth.getOrElse(createOAuthAuthorization())
-  
-  override def getReceiver(): NetworkReceiver[Status] = {
-    new TwitterReceiver(authorization, filters, storageLevel)
-  }
-}
-
-private[streaming]
-class TwitterReceiver(
-    twitterAuth: Authorization,
-    filters: Seq[String],
-    storageLevel: StorageLevel
-  ) extends NetworkReceiver[Status] {
-
-  var twitterStream: TwitterStream = _
-  lazy val blockGenerator = new BlockGenerator(storageLevel)
-
-  protected override def onStart() {
-    blockGenerator.start()
-    twitterStream = new TwitterStreamFactory().getInstance(twitterAuth)
-    twitterStream.addListener(new StatusListener {
-      def onStatus(status: Status) = {
-        blockGenerator += status
-      }
-      // Unimplemented
-      def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {}
-      def onTrackLimitationNotice(i: Int) {}
-      def onScrubGeo(l: Long, l1: Long) {}
-      def onStallWarning(stallWarning: StallWarning) {}
-      def onException(e: Exception) { stopOnError(e) }
-    })
-
-    val query: FilterQuery = new FilterQuery
-    if (filters.size > 0) {
-      query.track(filters.toArray)
-      twitterStream.filter(query)
-    } else {
-      twitterStream.sample()
-    }
-    logInfo("Twitter receiver started")
-  }
-
-  protected override def onStop() {
-    blockGenerator.stop()
-    twitterStream.shutdown()
-    logInfo("Twitter receiver stopped")
-  }
-}


[07/13] git commit: Merge branch 'apache-master' into project-refactor

Posted by pw...@apache.org.
Merge branch 'apache-master' into project-refactor


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/977bcc36
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/977bcc36
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/977bcc36

Branch: refs/heads/master
Commit: 977bcc36d4440ff562d5dbcc12449bf383d0d9e2
Parents: 87b915f 50e3b8e
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Dec 31 00:43:38 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Dec 31 00:43:38 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/TaskEndReason.scala  |  2 -
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  6 +--
 .../apache/spark/scheduler/DAGScheduler.scala   | 44 +++++++++++++-------
 .../org/apache/spark/scheduler/JobLogger.scala  |  6 +--
 .../scala/org/apache/spark/scheduler/Pool.scala |  4 --
 .../apache/spark/scheduler/Schedulable.scala    |  1 -
 .../apache/spark/scheduler/SparkListener.scala  | 20 +++++----
 .../spark/scheduler/SparkListenerBus.scala      |  2 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     |  7 ----
 .../apache/spark/scheduler/TaskSetManager.scala | 10 -----
 .../spark/ui/jobs/JobProgressListener.scala     | 11 ++---
 .../apache/spark/scheduler/JobLoggerSuite.scala |  2 +-
 .../spark/scheduler/SparkListenerSuite.scala    |  2 +-
 spark-class2.cmd                                |  2 +-
 14 files changed, 52 insertions(+), 67 deletions(-)
----------------------------------------------------------------------



[11/13] git commit: Removed XYZFunctions and added XYZUtils as a common Scala and Java interface for creating XYZ streams.

Posted by pw...@apache.org.
Removed XYZFunctions and added XYZUtils as a common Scala and Java interface for creating XYZ streams.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/aa99f226
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/aa99f226
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/aa99f226

Branch: refs/heads/master
Commit: aa99f226a691ddcb4442d60f4cd4908f434cc4ce
Parents: 3b4c4c7
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Jan 7 01:56:15 2014 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Jan 7 01:56:15 2014 -0800

----------------------------------------------------------------------
 .../streaming/examples/JavaFlumeEventCount.java |   5 +-
 .../streaming/examples/JavaKafkaWordCount.java  |   9 +-
 .../streaming/examples/FlumeEventCount.scala    |   2 +-
 .../streaming/examples/KafkaWordCount.scala     |   2 +-
 .../streaming/examples/MQTTWordCount.scala      |   2 +-
 .../streaming/examples/TwitterAlgebirdCMS.scala |   4 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |   2 +-
 .../streaming/examples/TwitterPopularTags.scala |   2 +-
 .../streaming/examples/ZeroMQWordCount.scala    |   3 +-
 .../api/java/flume/FlumeFunctions.scala         |  48 ------
 .../spark/streaming/flume/FlumeFunctions.scala  |  43 ------
 .../spark/streaming/flume/FlumeUtils.scala      |  70 +++++++++
 .../apache/spark/streaming/flume/package.scala  |  23 ---
 .../streaming/flume/JavaFlumeStreamSuite.java   |  13 +-
 .../streaming/flume/FlumeStreamSuite.scala      |   2 +-
 .../api/java/kafka/KafkaFunctions.scala         | 107 -------------
 .../spark/streaming/kafka/KafkaFunctions.scala  |  73 ---------
 .../spark/streaming/kafka/KafkaUtils.scala      | 153 +++++++++++++++++++
 .../apache/spark/streaming/kafka/package.scala  |  23 ---
 .../streaming/kafka/JavaKafkaStreamSuite.java   |  11 +-
 .../streaming/kafka/KafkaStreamSuite.scala      |   8 +-
 .../streaming/api/java/mqtt/MQTTFunctions.scala |  59 -------
 .../spark/streaming/mqtt/MQTTFunctions.scala    |  43 ------
 .../apache/spark/streaming/mqtt/MQTTUtils.scala |  75 +++++++++
 .../apache/spark/streaming/mqtt/package.scala   |  24 ---
 .../streaming/mqtt/JavaMQTTStreamSuite.java     |   6 +-
 .../spark/streaming/mqtt/MQTTStreamSuite.scala  |   4 +-
 .../api/java/twitter/TwitterFunctions.scala     |  99 ------------
 .../streaming/twitter/TwitterFunctions.scala    |  49 ------
 .../spark/streaming/twitter/TwitterUtils.scala  | 126 +++++++++++++++
 .../spark/streaming/twitter/package.scala       |  22 ---
 .../twitter/JavaTwitterStreamSuite.java         |  20 +--
 .../streaming/twitter/TwitterStreamSuite.scala  |  13 +-
 .../api/java/zeromq/ZeroMQFunctions.scala       | 102 -------------
 .../streaming/zeromq/ZeroMQFunctions.scala      |  57 -------
 .../spark/streaming/zeromq/ZeroMQUtils.scala    | 126 +++++++++++++++
 .../apache/spark/streaming/zeromq/package.scala |  24 ---
 .../streaming/zeromq/JavaZeroMQStreamSuite.java |  17 +--
 .../streaming/zeromq/ZeroMQStreamSuite.scala    |   8 +-
 39 files changed, 608 insertions(+), 871 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 83900a1..0a2b3de 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.examples;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.streaming.*;
 import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.streaming.api.java.flume.FlumeFunctions;
+import org.apache.spark.streaming.flume.FlumeUtils;
 import org.apache.spark.streaming.flume.SparkFlumeEvent;
 
 /**
@@ -53,8 +53,7 @@ public class JavaFlumeEventCount {
     JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
             System.getenv("SPARK_HOME"),
             JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class));
-    FlumeFunctions flumeFunc = new FlumeFunctions(ssc);
-    JavaDStream<SparkFlumeEvent> flumeStream = flumeFunc.flumeStream("localhost", port);
+    JavaDStream<SparkFlumeEvent> flumeStream = FlumeUtils.createStream(ssc, "localhost", port);
 
     flumeStream.count();
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 51de405..3bd7a3a 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.Duration;
 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.api.java.kafka.KafkaFunctions;
+import org.apache.spark.streaming.kafka.KafkaUtils;
 import scala.Tuple2;
 
 /**
@@ -54,7 +54,7 @@ public class JavaKafkaWordCount {
     }
 
     // Create the context with a 1 second batch size
-    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "KafkaWordCount",
+    JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount",
             new Duration(2000), System.getenv("SPARK_HOME"),
             JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class));
 
@@ -65,8 +65,7 @@ public class JavaKafkaWordCount {
       topicMap.put(topic, numThreads);
     }
 
-    KafkaFunctions kafkaFunc = new KafkaFunctions(ssc);
-    JavaPairDStream<String, String> messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap);
+    JavaPairDStream<String, String> messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap);
 
     JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
       @Override
@@ -96,6 +95,6 @@ public class JavaKafkaWordCount {
       });
 
     wordCounts.print();
-    ssc.start();
+    jssc.start();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
index 149640e..ae3709b 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
@@ -52,7 +52,7 @@ object FlumeEventCount {
       System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
 
     // Create a flume stream
-    val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
+    val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2)
 
     // Print out the count of events received from this server in each batch
     stream.count().map(cnt => "Received " + cnt + " flume events." ).print()

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 633712e..022c8c5 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -53,7 +53,7 @@ object KafkaWordCount {
     ssc.checkpoint("checkpoint")
 
     val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
-    val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2)
+    val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2)
     val words = lines.flatMap(_.split(" "))
     val wordCounts = words.map(x => (x, 1l))
       .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index f65c3f8..325290b 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -97,7 +97,7 @@ object MQTTWordCount {
 
     val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), 
     StreamingContext.jarOfClass(this.getClass))
-    val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY)
+    val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2)
 
     val words = lines.flatMap(x => x.toString.split(" "))
     val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index a60570f..3ccdc90 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -35,7 +35,7 @@ import org.apache.spark.streaming.twitter._
  * <p>
  * <p>
  *   <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
- *   This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure
+ *   This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a data structure
  *   for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc),
  *   that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the
  *   estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold
@@ -63,7 +63,7 @@ object TwitterAlgebirdCMS {
 
     val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10),
       System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
-    val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+    val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2)
 
     val users = stream.map(status => status.getUser.getId)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index 1382fa4..c7e83e7 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -50,7 +50,7 @@ object TwitterAlgebirdHLL {
 
     val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5),
       System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
-    val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+    val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER)
 
     val users = stream.map(status => status.getUser.getId)
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 84842b3..e2b0418 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -40,7 +40,7 @@ object TwitterPopularTags {
 
     val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2),
       System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
-    val stream = ssc.twitterStream(None, filters)
+    val stream = TwitterUtils.createStream(ssc, None, filters)
 
     val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index 789c5f2..5a76737 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -85,11 +85,10 @@ object ZeroMQWordCount {
     def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator
 
     //For this stream, a zeroMQ publisher should be running.
-    val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
+    val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _)
     val words = lines.flatMap(_.split(" "))
     val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
     wordCounts.print()
     ssc.start()
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
deleted file mode 100644
index 3347d19..0000000
--- a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.api.java.flume
-
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-import org.apache.spark.streaming.flume._
-import org.apache.spark.storage.StorageLevel
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Flume input streams.
- */
-class FlumeFunctions(javaStreamingContext: JavaStreamingContext) {
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   */
-  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
-    javaStreamingContext.ssc.flumeStream(hostname, port)
-  }
-
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
-    JavaDStream[SparkFlumeEvent] = {
-    javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
deleted file mode 100644
index 35e7a01..0000000
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.flume
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-/**
- * Extra Flume input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversion. Import org.apache.spark.streaming.flume._ to use these functions.
- */
-class FlumeFunctions(ssc: StreamingContext) {
-  /**
-   * Create a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream (
-      hostname: String,
-      port: Int,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[SparkFlumeEvent] = {
-    val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
new file mode 100644
index 0000000..834b775
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object FlumeUtils {
+  /**
+   * Create a input stream from a Flume source.
+   * @param ssc      StreamingContext object
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream (
+      ssc: StreamingContext,
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[SparkFlumeEvent] = {
+    val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      hostname: String,
+      port: Int
+    ): JavaDStream[SparkFlumeEvent] = {
+    createStream(jssc.ssc, hostname, port)
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel
+    ): JavaDStream[SparkFlumeEvent] = {
+    createStream(jssc.ssc, hostname, port, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
deleted file mode 100644
index c087a39..0000000
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object flume {
-  implicit def sscToFlumeFunctions(ssc: StreamingContext) = new FlumeFunctions(ssc)
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
index 5930fee..733389b 100644
--- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
+++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
@@ -1,4 +1,4 @@
-package org.apache.spark.streaming.flume;/*
+/*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
  * this work for additional information regarding copyright ownership.
@@ -15,21 +15,20 @@ package org.apache.spark.streaming.flume;/*
  * limitations under the License.
  */
 
+package org.apache.spark.streaming.flume;
+
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.LocalJavaStreamingContext;
 import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.flume.FlumeFunctions;
-import org.apache.spark.streaming.flume.SparkFlumeEvent;
+
 import org.junit.Test;
 
 public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
   @Test
   public void testFlumeStream() {
-    FlumeFunctions flumeFunc = new FlumeFunctions(ssc);
-
     // tests the API, does not actually test data receiving
-    JavaDStream<SparkFlumeEvent> test1 = flumeFunc.flumeStream("localhost", 12345);
-    JavaDStream<SparkFlumeEvent> test2 = flumeFunc.flumeStream("localhost", 12345,
+    JavaDStream<SparkFlumeEvent> test1 = FlumeUtils.createStream(ssc, "localhost", 12345);
+    JavaDStream<SparkFlumeEvent> test2 = FlumeUtils.createStream(ssc, "localhost", 12345,
       StorageLevel.MEMORY_AND_DISK_SER_2());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
index 74840f6..2e8e9fa 100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -39,7 +39,7 @@ class FlumeStreamSuite extends TestSuiteBase {
   test("flume input stream") {
     // Set up the streaming context and input streams
     val ssc = new StreamingContext(conf, batchDuration)
-    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+    val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK)
     val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
       with SynchronizedBuffer[Seq[SparkFlumeEvent]]
     val outputStream = new TestOutputStream(flumeStream, outputBuffer)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
deleted file mode 100644
index 491331b..0000000
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.api.java.kafka
-
-import scala.reflect.ClassTag
-import scala.collection.JavaConversions._
-
-import java.lang.{Integer => JInt}
-import java.util.{Map => JMap}
-
-import kafka.serializer.Decoder
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
-import org.apache.spark.streaming.kafka._
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Kafka input streams.
- */
-class KafkaFunctions(javaStreamingContext: JavaStreamingContext) {
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: JMap[String, JInt]
-    ): JavaPairDStream[String, String] = {
-      implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-      javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel RDD storage level.
-   *
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: JMap[String, JInt],
-      storageLevel: StorageLevel
-    ): JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param keyTypeClass Key type of RDD
-   * @param valueTypeClass value type of RDD
-   * @param keyDecoderClass Type of kafka key decoder
-   * @param valueDecoderClass Type of kafka value decoder
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   */
-  def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]](
-      keyTypeClass: Class[K],
-      valueTypeClass: Class[V],
-      keyDecoderClass: Class[U],
-      valueDecoderClass: Class[T],
-      kafkaParams: JMap[String, String],
-      topics: JMap[String, JInt],
-      storageLevel: StorageLevel
-    ): JavaPairDStream[K, V] = {
-    implicit val keyCmt: ClassTag[K] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
-    implicit val valueCmt: ClassTag[V] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
-
-    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
-    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
-
-    javaStreamingContext.ssc.kafkaStream[K, V, U, T](
-      kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
deleted file mode 100644
index 2135634..0000000
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.kafka
-
-import scala.reflect.ClassTag
-
-import kafka.serializer.{Decoder, StringDecoder}
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-/**
- * Extra Kafka input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversion. Import org.apache.spark.streaming.kafka._ to use these functions.
- */
-class KafkaFunctions(ssc: StreamingContext) {
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: Map[String, Int],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[(String, String)] = {
-    val kafkaParams = Map[String, String](
-      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
-      "zookeeper.connection.timeout.ms" -> "10000")
-    kafkaStream[String, String, StringDecoder, StringDecoder](
-      kafkaParams,
-      topics,
-      storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def kafkaStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest](
-      kafkaParams: Map[String, String],
-      topics: Map[String, Int],
-      storageLevel: StorageLevel
-    ): DStream[(K, V)] = {
-    val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
new file mode 100644
index 0000000..c2d851f
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.lang.{Integer => JInt}
+import java.util.{Map => JMap}
+
+import kafka.serializer.{Decoder, StringDecoder}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
+
+
+object KafkaUtils {
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param ssc       StreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
+   * @param groupId   The group id for this consumer
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def createStream(
+      ssc: StreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: Map[String, Int],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[(String, String)] = {
+    val kafkaParams = Map[String, String](
+      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
+      "zookeeper.connection.timeout.ms" -> "10000")
+    createStream[String, String, StringDecoder, StringDecoder](
+      ssc, kafkaParams, topics, storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param ssc         StreamingContext object
+   * @param kafkaParams Map of kafka configuration parameters,
+   *                    see http://kafka.apache.org/08/configuration.html
+   * @param topics      Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                    in its own thread.
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest](
+      ssc: StreamingContext,
+      kafkaParams: Map[String, String],
+      topics: Map[String, Int],
+      storageLevel: StorageLevel
+    ): DStream[(K, V)] = {
+    val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..)
+   * @param groupId   The group id for this consumer
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt]
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param zkQuorum  Zookeeper quorum (hostname:port,hostname:port,..).
+   * @param groupId   The group id for this consumer.
+   * @param topics    Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                  in its own thread.
+   * @param storageLevel RDD storage level.
+   *
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param jssc      JavaStreamingContext object
+   * @param keyTypeClass Key type of RDD
+   * @param valueTypeClass value type of RDD
+   * @param keyDecoderClass Type of kafka key decoder
+   * @param valueDecoderClass Type of kafka value decoder
+   * @param kafkaParams Map of kafka configuration parameters,
+   *                    see http://kafka.apache.org/08/configuration.html
+   * @param topics  Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *                in its own thread
+   * @param storageLevel RDD storage level. Defaults to MEMORY_AND_DISK_2.
+   */
+  def createStream[K, V, U <: Decoder[_], T <: Decoder[_]](
+      jssc: JavaStreamingContext,
+      keyTypeClass: Class[K],
+      valueTypeClass: Class[V],
+      keyDecoderClass: Class[U],
+      valueDecoderClass: Class[T],
+      kafkaParams: JMap[String, String],
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassTag[K] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val valueCmt: ClassTag[V] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+
+    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+    createStream[K, V, U, T](
+      jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
deleted file mode 100644
index 44e7ce6..0000000
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object kafka {
-  implicit def sscToKafkaFunctions(ssc: StreamingContext) = new KafkaFunctions(ssc)
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
index fdea96e..7b49994 100644
--- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -18,32 +18,27 @@
 package org.apache.spark.streaming.kafka;
 
 import java.util.HashMap;
-
-import org.apache.spark.streaming.api.java.kafka.KafkaFunctions;
 import org.junit.Test;
 import com.google.common.collect.Maps;
 import kafka.serializer.StringDecoder;
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.LocalJavaStreamingContext;
-import org.apache.spark.streaming.api.java.JavaDStream;
 import org.apache.spark.streaming.api.java.JavaPairDStream;
 
 public class JavaKafkaStreamSuite extends LocalJavaStreamingContext {
   @Test
   public void testKafkaStream() {
-
     HashMap<String, Integer> topics = Maps.newHashMap();
-    KafkaFunctions kafkaFunc = new KafkaFunctions(ssc);
 
     // tests the API, does not actually test data receiving
-    JavaPairDStream<String, String> test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics);
-    JavaPairDStream<String, String> test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics,
+    JavaPairDStream<String, String> test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics);
+    JavaPairDStream<String, String> test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics,
       StorageLevel.MEMORY_AND_DISK_SER_2());
 
     HashMap<String, String> kafkaParams = Maps.newHashMap();
     kafkaParams.put("zookeeper.connect", "localhost:12345");
     kafkaParams.put("group.id","consumer-group");
-    JavaPairDStream<String, String> test3 = kafkaFunc.kafkaStream(
+    JavaPairDStream<String, String> test3 = KafkaUtils.createStream(ssc,
       String.class, String.class, StringDecoder.class, StringDecoder.class,
       kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2());
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
index 2ef3e99..9c81f23 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -28,11 +28,11 @@ class KafkaStreamSuite extends TestSuiteBase {
     val topics = Map("my-topic" -> 1)
 
     // tests the API, does not actually test data receiving
-    val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
-    val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics)
+    val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2)
     val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
-    val test3 = ssc.kafkaStream[String, String, StringDecoder, StringDecoder](
-      kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+      ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
 
     // TODO: Actually test receiving data
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
deleted file mode 100644
index 7212495..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.api.java.mqtt
-
-import scala.reflect.ClassTag
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-import org.apache.spark.streaming.mqtt._
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating MQTT input streams.
- */
-class MQTTFunctions(javaStreamingContext: JavaStreamingContext) {
-
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String
-    ): JavaDStream[String] = {
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    javaStreamingContext.ssc.mqttStream(brokerUrl, topic)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level.
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): JavaDStream[String] = {
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
deleted file mode 100644
index 86f4e9c..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.mqtt
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-/**
- * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions.
- */
-class MQTTFunctions(ssc: StreamingContext) {
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[String] = {
-    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
new file mode 100644
index 0000000..0e6c25d
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+import scala.reflect.ClassTag
+
+object MQTTUtils {
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param ssc           StreamingContext object
+   * @param brokerUrl     Url of remote MQTT publisher
+   * @param topic         Topic name to subscribe to
+   * @param storageLevel  RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def createStream(
+      ssc: StreamingContext,
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[String] = {
+    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param jssc      JavaStreamingContext object
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic     Topic name to subscribe to
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      brokerUrl: String,
+      topic: String
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, brokerUrl, topic)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param jssc      JavaStreamingContext object
+   * @param brokerUrl     Url of remote MQTT publisher
+   * @param topic         Topic name to subscribe to
+   * @param storageLevel  RDD storage level.
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    createStream(jssc.ssc, brokerUrl, topic, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
deleted file mode 100644
index 28a944f..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object mqtt {
-  implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc)
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
index 3ddb4d0..44743aa 100644
--- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
@@ -19,7 +19,6 @@ package org.apache.spark.streaming.mqtt;
 
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions;
 import org.junit.Test;
 
 import org.apache.spark.streaming.LocalJavaStreamingContext;
@@ -29,11 +28,10 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext {
   public void testMQTTStream() {
     String brokerUrl = "abc";
     String topic = "def";
-    MQTTFunctions mqttFunc = new MQTTFunctions(ssc);
 
     // tests the API, does not actually test data receiving
-    JavaDStream<String> test1 = mqttFunc.mqttStream(brokerUrl, topic);
-    JavaDStream<String> test2 = mqttFunc.mqttStream(brokerUrl, topic,
+    JavaDStream<String> test1 = MQTTUtils.createStream(ssc, brokerUrl, topic);
+    JavaDStream<String> test2 = MQTTUtils.createStream(ssc, brokerUrl, topic,
       StorageLevel.MEMORY_AND_DISK_SER_2());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
index ab65429..fcc159e 100644
--- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -28,8 +28,8 @@ class MQTTStreamSuite extends TestSuiteBase {
     val topic = "def"
 
     // tests the API, does not actually test data receiving
-    val test1 = ssc.mqttStream(brokerUrl, topic)
-    val test2 = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic)
+    val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
 
     // TODO: Actually test receiving data
   }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
deleted file mode 100644
index 22e297a..0000000
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.api.java.twitter
-
-import twitter4j.Status
-import twitter4j.auth.Authorization
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-import org.apache.spark.streaming.twitter._
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Twitter input streams.
- */
-class TwitterFunctions(javaStreamingContext: JavaStreamingContext) {
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   */
-  def twitterStream(): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(None)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(filters: Array[String]): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(None, filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(None, filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   */
-  def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(Some(twitterAuth))
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization object
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
deleted file mode 100644
index e91049d..0000000
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.twitter
-
-import twitter4j.Status
-import twitter4j.auth.Authorization
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-/**
- * Extra Twitter input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversions. Import org.apache.spark.streaming.twitter._ to use these functions.
- */
-class TwitterFunctions(ssc: StreamingContext) {
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
-   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
-   *        twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   *        twitter4j.oauth.accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Option[Authorization],
-      filters: Seq[String] = Nil,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[Status] = {
-    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
new file mode 100644
index 0000000..5e506ff
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+object TwitterUtils {
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param ssc         StreamingContext object
+   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
+   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
+   *        twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   *        twitter4j.oauth.accessTokenSecret
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      ssc: StreamingContext,
+      twitterAuth: Option[Authorization],
+      filters: Seq[String] = Nil,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[Status] = {
+    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc   JavaStreamingContext object
+   */
+  def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = {
+    createStream(jssc.ssc, None)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc    JavaStreamingContext object
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = {
+    createStream(jssc.ssc, None, filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param jssc         JavaStreamingContext object
+   * @param filters      Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, None, filters, storageLevel)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc        JavaStreamingContext object
+   * @param twitterAuth Twitter4J Authorization
+   */
+  def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth))
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc        JavaStreamingContext object
+   * @param twitterAuth Twitter4J Authorization
+   * @param filters     Set of filter strings to get only those tweets that match them
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      twitterAuth: Authorization,
+      filters: Array[String]
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth), filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param jssc         JavaStreamingContext object
+   * @param twitterAuth  Twitter4J Authorization object
+   * @param filters      Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def createStream(
+      jssc: JavaStreamingContext,
+      twitterAuth: Authorization,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
deleted file mode 100644
index 23f82c5..0000000
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object twitter {
-  implicit def sscToTwitterFunctions(ssc: StreamingContext) = new TwitterFunctions(ssc)
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
index 4564d6c..e46b4e5 100644
--- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
@@ -19,13 +19,10 @@ package org.apache.spark.streaming.twitter;
 
 import java.util.Arrays;
 
-import org.apache.spark.streaming.api.java.twitter.TwitterFunctions;
 import org.junit.Test;
-
 import twitter4j.Status;
 import twitter4j.auth.Authorization;
 import twitter4j.auth.NullAuthorization;
-
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.LocalJavaStreamingContext;
 import org.apache.spark.streaming.api.java.JavaDStream;
@@ -33,18 +30,17 @@ import org.apache.spark.streaming.api.java.JavaDStream;
 public class JavaTwitterStreamSuite extends LocalJavaStreamingContext {
   @Test
   public void testTwitterStream() {
-    TwitterFunctions twitterFunc = new TwitterFunctions(ssc);
     String[] filters = (String[])Arrays.<String>asList("filter1", "filter2").toArray();
     Authorization auth = NullAuthorization.getInstance();
 
     // tests the API, does not actually test data receiving
-    JavaDStream<Status> test1 = twitterFunc.twitterStream();
-    JavaDStream<Status> test2 = twitterFunc.twitterStream(filters);
-    JavaDStream<Status> test3 =
-      twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2());
-    JavaDStream<Status> test4 = twitterFunc.twitterStream(auth);
-    JavaDStream<Status> test5 = twitterFunc.twitterStream(auth, filters);
-    JavaDStream<Status> test6 =
-      twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<Status> test1 = TwitterUtils.createStream(ssc);
+    JavaDStream<Status> test2 = TwitterUtils.createStream(ssc, filters);
+    JavaDStream<Status> test3 = TwitterUtils.createStream(
+      ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<Status> test4 = TwitterUtils.createStream(ssc, auth);
+    JavaDStream<Status> test5 = TwitterUtils.createStream(ssc, auth, filters);
+    JavaDStream<Status> test6 = TwitterUtils.createStream(ssc,
+      auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
index d7f6d35..a0a8fe6 100644
--- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -29,12 +29,13 @@ class TwitterStreamSuite extends TestSuiteBase {
     val authorization: Authorization = NullAuthorization.getInstance()
 
     // tests the API, does not actually test data receiving
-    val test1 = ssc.twitterStream(None)
-    val test2 = ssc.twitterStream(None, filters)
-    val test3 = ssc.twitterStream(None, filters, StorageLevel.MEMORY_AND_DISK_SER_2)
-    val test4 = ssc.twitterStream(Some(authorization))
-    val test5 = ssc.twitterStream(Some(authorization), filters)
-    val test6 = ssc.twitterStream(Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test1 = TwitterUtils.createStream(ssc, None)
+    val test2 = TwitterUtils.createStream(ssc, None, filters)
+    val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test4 = TwitterUtils.createStream(ssc, Some(authorization))
+    val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters)
+    val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters,
+      StorageLevel.MEMORY_AND_DISK_SER_2)
 
     // Note that actually testing the data receiving is hard as authentication keys are
     // necessary for accessing Twitter live stream

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
deleted file mode 100644
index a9bbce7..0000000
--- a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.api.java.zeromq
-
-import scala.reflect.ClassTag
-import scala.collection.JavaConversions._
-
-import akka.actor.SupervisorStrategy
-import akka.util.ByteString
-import akka.zeromq.Subscribe
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.api.java.function.{Function => JFunction}
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-import org.apache.spark.streaming.zeromq._
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating ZeroMQ input streams.
- */
-class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) {
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote ZeroMQ publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel,
-      supervisorStrategy: SupervisorStrategy
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level.
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
-  }
-}


[03/13] git commit: Refactored kafka, flume, zeromq, mqtt as separate external projects, with their own self-contained scala API, java API, scala unit tests and java unit tests. Updated examples to use the external projects.

Posted by pw...@apache.org.
Refactored kafka, flume, zeromq, mqtt as separate external projects, with their own self-contained scala API, java API, scala unit tests and java unit tests. Updated examples to use the external projects.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/f4e40661
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/f4e40661
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/f4e40661

Branch: refs/heads/master
Commit: f4e40661912af2a23e250a49f72f00675172e2de
Parents: 6e43039
Author: Tathagata Das <ta...@gmail.com>
Authored: Mon Dec 30 11:13:24 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Mon Dec 30 11:13:24 2013 -0800

----------------------------------------------------------------------
 .../streaming/examples/JavaFlumeEventCount.java |  11 +-
 .../streaming/examples/JavaKafkaWordCount.java  |   4 +-
 .../streaming/examples/FlumeEventCount.scala    |   1 +
 .../streaming/examples/KafkaWordCount.scala     |   1 +
 .../streaming/examples/MQTTWordCount.scala      |  10 +-
 .../streaming/examples/ZeroMQWordCount.scala    |   6 +-
 .../spark/streaming/flume/FlumeFunctions.scala  |  43 ++++
 .../streaming/flume/FlumeInputDStream.scala     | 155 ++++++++++++
 .../flume/JavaStreamingContextWithFlume.scala   |  48 ++++
 .../apache/spark/streaming/flume/package.scala  |  23 ++
 .../src/test/java/JavaFlumeStreamSuite.java     |  38 +++
 .../flume/src/test/resources/log4j.properties   |  29 +++
 .../streaming/flume/FlumeStreamSuite.scala      |  86 +++++++
 .../kafka/JavaStreamingContextWithKafka.scala   | 107 ++++++++
 .../spark/streaming/kafka/KafkaFunctions.scala  |  73 ++++++
 .../streaming/kafka/KafkaInputDStream.scala     | 155 ++++++++++++
 .../apache/spark/streaming/kafka/package.scala  |  23 ++
 .../streaming/kafka/JavaKafkaStreamSuite.java   |  51 ++++
 .../kafka/src/test/resources/log4j.properties   |  29 +++
 .../streaming/kafka/KafkaStreamSuite.scala      |  39 +++
 .../mqtt/JavaStreamingContextWithMQTT.scala     |  59 +++++
 .../spark/streaming/mqtt/MQTTFunctions.scala    |  43 ++++
 .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 +++++++++
 .../spark/spark/streaming/mqtt/package.scala    |  24 ++
 .../streaming/mqtt/JavaMQTTStreamSuite.java     |  41 ++++
 .../mqtt/src/test/resources/log4j.properties    |  29 +++
 .../spark/streaming/mqtt/MQTTStreamSuite.scala  |  36 +++
 .../JavaStreamingContextWithTwitter.scala       |  99 ++++++++
 .../twitter/StreamingContextWithTwitter.scala   |  27 --
 .../streaming/twitter/TwitterFunctions.scala    |  49 ++++
 .../spark/streaming/twitter/package.scala       |  21 +-
 .../twitter/JavaTwitterStreamSuite.java         |  51 ++++
 .../twitter/src/test/resources/log4j.properties |  29 +++
 .../streaming/twitter/TwitterStreamSuite.scala  |  42 ++++
 .../zeromq/JavaStreamingContextWithZeroMQ.scala | 102 ++++++++
 .../streaming/zeromq/ZeroMQFunctions.scala      |  57 +++++
 .../spark/streaming/zeromq/ZeroMQReceiver.scala |  54 ++++
 .../apache/spark/streaming/zeromq/package.scala |  24 ++
 .../streaming/zeromq/JavaZeroMQStreamSuite.java |  55 +++++
 .../zeromq/src/test/resources/log4j.properties  |  29 +++
 .../streaming/zeromq/ZeroMQStreamSuite.scala    |  44 ++++
 project/SparkBuild.scala                        |  89 +++++--
 .../spark/streaming/StreamingContext.scala      | 129 +---------
 .../api/java/JavaStreamingContext.scala         | 244 +------------------
 .../streaming/dstream/FlumeInputDStream.scala   | 154 ------------
 .../streaming/dstream/KafkaInputDStream.scala   | 153 ------------
 .../streaming/dstream/MQTTInputDStream.scala    | 110 ---------
 .../streaming/receivers/ZeroMQReceiver.scala    |  53 ----
 .../streaming/scheduler/JobGenerator.scala      |   1 +
 .../scheduler/NetworkInputTracker.scala         |   5 +-
 .../apache/spark/streaming/JavaAPISuite.java    |  80 +-----
 .../streaming/LocalJavaStreamingContext.java    |  46 ++++
 .../spark/streaming/InputStreamsSuite.scala     |  75 +-----
 .../apache/spark/streaming/TestSuiteBase.scala  |   9 +-
 54 files changed, 2059 insertions(+), 1046 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 261813b..e53c4f9 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -20,7 +20,8 @@ package org.apache.spark.streaming.examples;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.streaming.*;
 import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.streaming.dstream.SparkFlumeEvent;
+import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume;
+import org.apache.spark.streaming.flume.SparkFlumeEvent;
 
 /**
  *  Produces a count of events received from Flume.
@@ -49,10 +50,10 @@ public class JavaFlumeEventCount {
 
     Duration batchInterval = new Duration(2000);
 
-    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
+    JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
             System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
-
-    JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
+    JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc);
+    JavaDStream<SparkFlumeEvent> flumeStream = sscWithFlume.flumeStream("localhost", port);
 
     flumeStream.count();
 
@@ -63,6 +64,6 @@ public class JavaFlumeEventCount {
       }
     }).print();
 
-    sc.start();
+    ssc.start();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 22994fb..de0420c 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -29,6 +29,7 @@ import org.apache.spark.streaming.Duration;
 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.kafka.JavaStreamingContextWithKafka;
 import scala.Tuple2;
 
 /**
@@ -63,7 +64,8 @@ public class JavaKafkaWordCount {
       topicMap.put(topic, numThreads);
     }
 
-    JavaPairDStream<String, String> messages = ssc.kafkaStream(args[1], args[2], topicMap);
+    JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc);
+    JavaPairDStream<String, String> messages = sscWithKafka.kafkaStream(args[1], args[2], topicMap);
 
     JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
index 9f6e163..f1641e2 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
 import org.apache.spark.util.IntParam
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming._
+import org.apache.spark.streaming.flume._
 
 /**
  *  Produces a count of events received from Flume.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 570ba4c..833c83a 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -24,6 +24,7 @@ import kafka.producer._
 import org.apache.spark.streaming._
 import org.apache.spark.streaming.StreamingContext._
 import org.apache.spark.streaming.util.RawTextHelper._
+import org.apache.spark.streaming.kafka._
 
 /**
  * Consumes messages from one or more topics in Kafka and does wordcount.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index ff332a0..edb46ac 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -17,11 +17,6 @@
 
 package org.apache.spark.streaming.examples
 
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.dstream.MQTTReceiver
-import org.apache.spark.storage.StorageLevel
-
 import org.eclipse.paho.client.mqttv3.MqttClient
 import org.eclipse.paho.client.mqttv3.MqttClientPersistence
 import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
@@ -29,6 +24,11 @@ import org.eclipse.paho.client.mqttv3.MqttException
 import org.eclipse.paho.client.mqttv3.MqttMessage
 import org.eclipse.paho.client.mqttv3.MqttTopic
 
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.mqtt._
+
 /**
  * A simple Mqtt publisher for demonstration purposes, repeatedly publishes 
  * Space separated String Message "hello mqtt demo for spark streaming"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index e83ce78..037b96e 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -20,11 +20,13 @@ package org.apache.spark.streaming.examples
 import akka.actor.ActorSystem
 import akka.actor.actorRef2Scala
 import akka.zeromq._
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
 import akka.zeromq.Subscribe
 import akka.util.ByteString
 
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.zeromq._
+
 /**
  * A simple publisher for demonstration purposes, repeatedly publishes random Messages
  * every one second.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
new file mode 100644
index 0000000..35e7a01
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeFunctions.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+/**
+ * Extra Flume input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversion. Import org.apache.spark.streaming.flume._ to use these functions.
+ */
+class FlumeFunctions(ssc: StreamingContext) {
+  /**
+   * Create a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def flumeStream (
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[SparkFlumeEvent] = {
+    val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
new file mode 100644
index 0000000..ce3ef47
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import java.net.InetSocketAddress
+import java.io.{ObjectInput, ObjectOutput, Externalizable}
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import org.apache.flume.source.avro.AvroSourceProtocol
+import org.apache.flume.source.avro.AvroFlumeEvent
+import org.apache.flume.source.avro.Status
+import org.apache.avro.ipc.specific.SpecificResponder
+import org.apache.avro.ipc.NettyServer
+
+import org.apache.spark.util.Utils
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
+
+private[streaming]
+class FlumeInputDStream[T: ClassTag](
+  @transient ssc_ : StreamingContext,
+  host: String,
+  port: Int,
+  storageLevel: StorageLevel
+) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
+
+  override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = {
+    new FlumeReceiver(host, port, storageLevel)
+  }
+}
+
+/**
+ * A wrapper class for AvroFlumeEvent's with a custom serialization format.
+ *
+ * This is necessary because AvroFlumeEvent uses inner data structures
+ * which are not serializable.
+ */
+class SparkFlumeEvent() extends Externalizable {
+  var event : AvroFlumeEvent = new AvroFlumeEvent()
+
+  /* De-serialize from bytes. */
+  def readExternal(in: ObjectInput) {
+    val bodyLength = in.readInt()
+    val bodyBuff = new Array[Byte](bodyLength)
+    in.read(bodyBuff)
+
+    val numHeaders = in.readInt()
+    val headers = new java.util.HashMap[CharSequence, CharSequence]
+
+    for (i <- 0 until numHeaders) {
+      val keyLength = in.readInt()
+      val keyBuff = new Array[Byte](keyLength)
+      in.read(keyBuff)
+      val key : String = Utils.deserialize(keyBuff)
+
+      val valLength = in.readInt()
+      val valBuff = new Array[Byte](valLength)
+      in.read(valBuff)
+      val value : String = Utils.deserialize(valBuff)
+
+      headers.put(key, value)
+    }
+
+    event.setBody(ByteBuffer.wrap(bodyBuff))
+    event.setHeaders(headers)
+  }
+
+  /* Serialize to bytes. */
+  def writeExternal(out: ObjectOutput) {
+    val body = event.getBody.array()
+    out.writeInt(body.length)
+    out.write(body)
+
+    val numHeaders = event.getHeaders.size()
+    out.writeInt(numHeaders)
+    for ((k, v) <- event.getHeaders) {
+      val keyBuff = Utils.serialize(k.toString)
+      out.writeInt(keyBuff.length)
+      out.write(keyBuff)
+      val valBuff = Utils.serialize(v.toString)
+      out.writeInt(valBuff.length)
+      out.write(valBuff)
+    }
+  }
+}
+
+private[streaming] object SparkFlumeEvent {
+  def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = {
+    val event = new SparkFlumeEvent
+    event.event = in
+    event
+  }
+}
+
+/** A simple server that implements Flume's Avro protocol. */
+private[streaming]
+class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol {
+  override def append(event : AvroFlumeEvent) : Status = {
+    receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)
+    Status.OK
+  }
+
+  override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = {
+    events.foreach (event =>
+      receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event))
+    Status.OK
+  }
+}
+
+/** A NetworkReceiver which listens for events using the
+  * Flume Avro interface.*/
+private[streaming]
+class FlumeReceiver(
+    host: String,
+    port: Int,
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[SparkFlumeEvent] {
+
+  lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+  protected override def onStart() {
+    val responder = new SpecificResponder(
+      classOf[AvroSourceProtocol], new FlumeEventServer(this))
+    val server = new NettyServer(responder, new InetSocketAddress(host, port))
+    blockGenerator.start()
+    server.start()
+    logInfo("Flume receiver started")
+  }
+
+  protected override def onStop() {
+    blockGenerator.stop()
+    logInfo("Flume receiver stopped")
+  }
+
+  override def getLocationPreference = Some(host)
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
new file mode 100644
index 0000000..4e66ae3
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Flume input streams.
+ */
+class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext)
+  extends JavaStreamingContext(javaStreamingContext.ssc) {
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   */
+  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
+    ssc.flumeStream(hostname, port)
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
+    JavaDStream[SparkFlumeEvent] = {
+    ssc.flumeStream(hostname, port, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
new file mode 100644
index 0000000..c087a39
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+package object flume {
+  implicit def sscToFlumeFunctions(ssc: StreamingContext) = new FlumeFunctions(ssc)
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/test/java/JavaFlumeStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/flume/src/test/java/JavaFlumeStreamSuite.java b/external/flume/src/test/java/JavaFlumeStreamSuite.java
new file mode 100644
index 0000000..deffc78
--- /dev/null
+++ b/external/flume/src/test/java/JavaFlumeStreamSuite.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume;
+import org.apache.spark.streaming.flume.SparkFlumeEvent;
+import org.junit.Test;
+
+public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testFlumeStream() {
+    JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc);
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<SparkFlumeEvent> test1 = sscWithFlume.flumeStream("localhost", 12345);
+    JavaDStream<SparkFlumeEvent> test2 = sscWithFlume.flumeStream("localhost", 12345,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    // To verify that JavaStreamingContextWithKafka is also StreamingContext
+    JavaDStream<String> socketStream = sscWithFlume.socketTextStream("localhost", 9999);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties
new file mode 100644
index 0000000..063529a
--- /dev/null
+++ b/external/flume/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
new file mode 100644
index 0000000..ba33320
--- /dev/null
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
+
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+import java.nio.charset.Charset
+
+import org.apache.avro.ipc.NettyTransceiver
+import org.apache.avro.ipc.specific.SpecificRequestor
+import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase}
+import org.apache.spark.streaming.util.ManualClock
+
+class FlumeStreamSuite extends TestSuiteBase {
+
+  val testPort = 9999
+
+  test("flume input stream") {
+    // Set up the streaming context and input streams
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
+      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
+    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val input = Seq(1, 2, 3, 4, 5)
+    Thread.sleep(1000)
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
+    val client = SpecificRequestor.getClient(
+      classOf[AvroSourceProtocol], transceiver)
+
+    for (i <- 0 until input.size) {
+      val event = new AvroFlumeEvent
+      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
+      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
+      client.append(event)
+      Thread.sleep(500)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+
+    val startTime = System.currentTimeMillis()
+    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
+      Thread.sleep(100)
+    }
+    Thread.sleep(1000)
+    val timeTaken = System.currentTimeMillis() - startTime
+    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+    logInfo("Stopping context")
+    ssc.stop()
+
+    val decoder = Charset.forName("UTF-8").newDecoder()
+
+    assert(outputBuffer.size === input.length)
+    for (i <- 0 until outputBuffer.size) {
+      assert(outputBuffer(i).size === 1)
+      val str = decoder.decode(outputBuffer(i).head.event.getBody)
+      assert(str.toString === input(i).toString)
+      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
new file mode 100644
index 0000000..ab0e8a6
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.lang.{Integer => JInt}
+import java.util.{Map => JMap}
+
+import kafka.serializer.Decoder
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Kafka input streams.
+ */
+class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext)
+  extends JavaStreamingContext(javaStreamingContext.ssc) {
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   */
+  def kafkaStream(
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt]
+    ): JavaPairDStream[String, String] = {
+      implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+      ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *               in its own thread.
+   * @param storageLevel RDD storage level.
+   *
+   */
+  def kafkaStream(
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param keyTypeClass Key type of RDD
+   * @param valueTypeClass value type of RDD
+   * @param keyDecoderClass Type of kafka key decoder
+   * @param valueDecoderClass Type of kafka value decoder
+   * @param kafkaParams Map of kafka configuration paramaters.
+   *                    See: http://kafka.apache.org/configuration.html
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   * @param storageLevel RDD storage level. Defaults to memory-only
+   */
+  def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]](
+      keyTypeClass: Class[K],
+      valueTypeClass: Class[V],
+      keyDecoderClass: Class[U],
+      valueDecoderClass: Class[T],
+      kafkaParams: JMap[String, String],
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassTag[K] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val valueCmt: ClassTag[V] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+
+    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+    ssc.kafkaStream[K, V, U, T](
+      kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
new file mode 100644
index 0000000..2135634
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaFunctions.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.reflect.ClassTag
+
+import kafka.serializer.{Decoder, StringDecoder}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+/**
+ * Extra Kafka input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversion. Import org.apache.spark.streaming.kafka._ to use these functions.
+ */
+class KafkaFunctions(ssc: StreamingContext) {
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *               in its own thread.
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def kafkaStream(
+      zkQuorum: String,
+      groupId: String,
+      topics: Map[String, Int],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[(String, String)] = {
+    val kafkaParams = Map[String, String](
+      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
+      "zookeeper.connection.timeout.ms" -> "10000")
+    kafkaStream[String, String, StringDecoder, StringDecoder](
+      kafkaParams,
+      topics,
+      storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages from a Kafka Broker.
+   * @param kafkaParams Map of kafka configuration paramaters.
+   *                    See: http://kafka.apache.org/configuration.html
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *               in its own thread.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def kafkaStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest](
+      kafkaParams: Map[String, String],
+      topics: Map[String, Int],
+      storageLevel: StorageLevel
+    ): DStream[(K, V)] = {
+    val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
new file mode 100644
index 0000000..fd69328
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.collection.Map
+import scala.reflect.ClassTag
+
+import java.util.Properties
+import java.util.concurrent.Executors
+
+import kafka.consumer._
+import kafka.serializer.Decoder
+import kafka.utils.VerifiableProperties
+import kafka.utils.ZKStringSerializer
+import org.I0Itec.zkclient._
+
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
+
+
+/**
+ * Input stream that pulls messages from a Kafka Broker.
+ *
+ * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param storageLevel RDD storage level.
+ */
+private[streaming]
+class KafkaInputDStream[
+  K: ClassTag,
+  V: ClassTag,
+  U <: Decoder[_]: Manifest,
+  T <: Decoder[_]: Manifest](
+    @transient ssc_ : StreamingContext,
+    kafkaParams: Map[String, String],
+    topics: Map[String, Int],
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[(K, V)](ssc_) with Logging {
+
+  def getReceiver(): NetworkReceiver[(K, V)] = {
+    new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
+        .asInstanceOf[NetworkReceiver[(K, V)]]
+  }
+}
+
+private[streaming]
+class KafkaReceiver[
+  K: ClassTag,
+  V: ClassTag,
+  U <: Decoder[_]: Manifest,
+  T <: Decoder[_]: Manifest](
+    kafkaParams: Map[String, String],
+    topics: Map[String, Int],
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[Any] {
+
+  // Handles pushing data into the BlockManager
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+  // Connection to Kafka
+  var consumerConnector : ConsumerConnector = null
+
+  def onStop() {
+    blockGenerator.stop()
+  }
+
+  def onStart() {
+
+    blockGenerator.start()
+
+    // In case we are using multiple Threads to handle Kafka Messages
+    val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
+
+    logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id"))
+
+    // Kafka connection properties
+    val props = new Properties()
+    kafkaParams.foreach(param => props.put(param._1, param._2))
+
+    // Create the connection to the cluster
+    logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect"))
+    val consumerConfig = new ConsumerConfig(props)
+    consumerConnector = Consumer.create(consumerConfig)
+    logInfo("Connected to " + kafkaParams("zookeeper.connect"))
+
+    // When autooffset.reset is defined, it is our responsibility to try and whack the
+    // consumer group zk node.
+    if (kafkaParams.contains("auto.offset.reset")) {
+      tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id"))
+    }
+
+    val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[K]]
+    val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
+      .newInstance(consumerConfig.props)
+      .asInstanceOf[Decoder[V]]
+
+    // Create Threads for each Topic/Message Stream we are listening
+    val topicMessageStreams = consumerConnector.createMessageStreams(
+      topics, keyDecoder, valueDecoder)
+
+
+    // Start the messages handler for each partition
+    topicMessageStreams.values.foreach { streams =>
+      streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
+    }
+  }
+
+  // Handles Kafka Messages
+  private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V])
+    extends Runnable {
+    def run() {
+      logInfo("Starting MessageHandler.")
+      for (msgAndMetadata <- stream) {
+        blockGenerator += (msgAndMetadata.key, msgAndMetadata.message)
+      }
+    }
+  }
+
+  // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because
+  // Kafka 0.7.2 only honors this param when the group is not in zookeeper.
+  //
+  // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas'
+  // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest':
+  // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
+  private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) {
+    try {
+      val dir = "/consumers/" + groupId
+      logInfo("Cleaning up temporary zookeeper data under " + dir + ".")
+      val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer)
+      zk.deleteRecursive(dir)
+      zk.close()
+    } catch {
+      case _ : Throwable => // swallow
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
new file mode 100644
index 0000000..44e7ce6
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+package object kafka {
+  implicit def sscToKafkaFunctions(ssc: StreamingContext) = new KafkaFunctions(ssc)
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
new file mode 100644
index 0000000..66236df
--- /dev/null
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka;
+
+import java.util.HashMap;
+import org.junit.Test;
+import com.google.common.collect.Maps;
+import kafka.serializer.StringDecoder;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+
+public class JavaKafkaStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testKafkaStream() {
+
+    HashMap<String, Integer> topics = Maps.newHashMap();
+    JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc);
+
+    // tests the API, does not actually test data receiving
+    JavaPairDStream<String, String> test1 = sscWithKafka.kafkaStream("localhost:12345", "group", topics);
+    JavaPairDStream<String, String> test2 = sscWithKafka.kafkaStream("localhost:12345", "group", topics,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    HashMap<String, String> kafkaParams = Maps.newHashMap();
+    kafkaParams.put("zookeeper.connect","localhost:12345");
+    kafkaParams.put("group.id","consumer-group");
+    JavaPairDStream<String, String> test3 = sscWithKafka.kafkaStream(
+      String.class, String.class, StringDecoder.class, StringDecoder.class,
+      kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    // To verify that JavaStreamingContextWithKafka is also StreamingContext
+    JavaDStream<String> socketStream = sscWithKafka.socketTextStream("localhost", 9999);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties
new file mode 100644
index 0000000..063529a
--- /dev/null
+++ b/external/kafka/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
new file mode 100644
index 0000000..2ef3e99
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import kafka.serializer.StringDecoder
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class KafkaStreamSuite extends TestSuiteBase {
+
+  test("kafka input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val topics = Map("my-topic" -> 1)
+
+    // tests the API, does not actually test data receiving
+    val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
+    val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
+    val test3 = ssc.kafkaStream[String, String, StringDecoder, StringDecoder](
+      kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // TODO: Actually test receiving data
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
new file mode 100644
index 0000000..d814da0
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating MQTT input streams.
+ */
+class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext)
+  extends JavaStreamingContext(javaStreamingContext.ssc) {
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    ssc.mqttStream(brokerUrl, topic)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   * @param storageLevel RDD storage level.
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    ssc.mqttStream(brokerUrl, topic, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
new file mode 100644
index 0000000..86f4e9c
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+/**
+ * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions.
+ */
+class MQTTFunctions(ssc: StreamingContext) {
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[String] = {
+    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
new file mode 100644
index 0000000..c8987a3
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import java.util.Properties
+import java.util.concurrent.Executors
+import java.io.IOException
+
+import org.eclipse.paho.client.mqttv3.MqttCallback
+import org.eclipse.paho.client.mqttv3.MqttClient
+import org.eclipse.paho.client.mqttv3.MqttClientPersistence
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
+import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
+import org.eclipse.paho.client.mqttv3.MqttException
+import org.eclipse.paho.client.mqttv3.MqttMessage
+import org.eclipse.paho.client.mqttv3.MqttTopic
+
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
+
+/**
+ * Input stream that subscribe messages from a Mqtt Broker.
+ * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
+ * @param brokerUrl Url of remote mqtt publisher
+ * @param topic topic name to subscribe to
+ * @param storageLevel RDD storage level.
+ */
+
+private[streaming] 
+class MQTTInputDStream[T: ClassTag](
+    @transient ssc_ : StreamingContext,
+    brokerUrl: String,
+    topic: String,
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_) with Logging {
+  
+  def getReceiver(): NetworkReceiver[T] = {
+    new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]]
+  }
+}
+
+private[streaming] 
+class MQTTReceiver(brokerUrl: String,
+  topic: String,
+  storageLevel: StorageLevel
+  ) extends NetworkReceiver[Any] {
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+  
+  def onStop() {
+    blockGenerator.stop()
+  }
+  
+  def onStart() {
+
+    blockGenerator.start()
+
+    // Set up persistence for messages 
+    var peristance: MqttClientPersistence = new MemoryPersistence()
+
+    // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
+    var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance)
+
+    // Connect to MqttBroker    
+    client.connect()
+
+    // Subscribe to Mqtt topic
+    client.subscribe(topic)
+
+    // Callback automatically triggers as and when new message arrives on specified topic
+    var callback: MqttCallback = new MqttCallback() {
+
+      // Handles Mqtt message 
+      override def messageArrived(arg0: String, arg1: MqttMessage) {
+        blockGenerator += new String(arg1.getPayload())
+      }
+
+      override def deliveryComplete(arg0: IMqttDeliveryToken) {
+      }
+
+      override def connectionLost(arg0: Throwable) {
+        logInfo("Connection lost " + arg0)
+      }
+    }
+
+    // Set up callback for MqttClient
+    client.setCallback(callback)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
new file mode 100644
index 0000000..28a944f
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+package object mqtt {
+  implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc)
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
new file mode 100644
index 0000000..c1f4164
--- /dev/null
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.junit.Test;
+
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+
+public class JavaMQTTStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testMQTTStream() {
+    String brokerUrl = "abc";
+    String topic = "def";
+    JavaStreamingContextWithMQTT sscWithMQTT = new JavaStreamingContextWithMQTT(ssc);
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<String> test1 = sscWithMQTT.mqttStream(brokerUrl, topic);
+    JavaDStream<String> test2 = sscWithMQTT.mqttStream(brokerUrl, topic,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    // To verify that JavaStreamingContextWithKafka is also StreamingContext
+    JavaDStream<String> socketStream = sscWithMQTT.socketTextStream("localhost", 9999);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties
new file mode 100644
index 0000000..063529a
--- /dev/null
+++ b/external/mqtt/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
new file mode 100644
index 0000000..ab65429
--- /dev/null
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class MQTTStreamSuite extends TestSuiteBase {
+
+  test("MQTT input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val brokerUrl = "abc"
+    val topic = "def"
+
+    // tests the API, does not actually test data receiving
+    val test1 = ssc.mqttStream(brokerUrl, topic)
+    val test2 = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // TODO: Actually test receiving data
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
new file mode 100644
index 0000000..0250364
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Twitter input streams.
+ */
+class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext)
+  extends JavaStreamingContext(javaStreamingContext.ssc) {
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   */
+  def twitterStream(): JavaDStream[Status] = {
+    ssc.twitterStream(None)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def twitterStream(filters: Array[String]): JavaDStream[Status] = {
+    ssc.twitterStream(None, filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = {
+    ssc.twitterStream(None, filters, storageLevel)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization
+   */
+  def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = {
+    ssc.twitterStream(Some(twitterAuth))
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def twitterStream(
+      twitterAuth: Authorization,
+      filters: Array[String]
+    ): JavaDStream[Status] = {
+    ssc.twitterStream(Some(twitterAuth), filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization object
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      twitterAuth: Authorization,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
deleted file mode 100644
index fe66e28..0000000
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/StreamingContextWithTwitter.scala
+++ /dev/null
@@ -1,27 +0,0 @@
-package org.apache.spark.streaming.twitter
-
-import twitter4j.Status
-import twitter4j.auth.Authorization
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-
-class StreamingContextWithTwitter(ssc: StreamingContext) {
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
-   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
-   *        .consumerSecret, .accessToken and .accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Option[Authorization] = None,
-      filters: Seq[String] = Nil,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[Status] = {
-    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
new file mode 100644
index 0000000..e91049d
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterFunctions.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+/**
+ * Extra Twitter input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversions. Import org.apache.spark.streaming.twitter._ to use these functions.
+ */
+class TwitterFunctions(ssc: StreamingContext) {
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
+   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
+   *        twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   *        twitter4j.oauth.accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      twitterAuth: Option[Authorization],
+      filters: Seq[String] = Nil,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[Status] = {
+    val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
index 89c202a..23f82c5 100644
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala
@@ -1,7 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.spark.streaming
 
 package object twitter {
-  implicit def enrichMyStreamingContext(ssc: StreamingContext): StreamingContextWithTwitter = {
-    new StreamingContextWithTwitter(ssc)
-  }
+  implicit def sscToTwitterFunctions(ssc: StreamingContext) = new TwitterFunctions(ssc)
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
new file mode 100644
index 0000000..34e4fbd
--- /dev/null
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter;
+
+import java.util.Arrays;
+import org.junit.Test;
+
+import twitter4j.Status;
+import twitter4j.auth.Authorization;
+import twitter4j.auth.NullAuthorization;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaTwitterStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testTwitterStream() {
+    JavaStreamingContextWithTwitter sscWithTwitter = new JavaStreamingContextWithTwitter(ssc);
+    String[] filters = (String[])Arrays.<String>asList("filter1", "filter2").toArray();
+    Authorization auth = NullAuthorization.getInstance();
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<Status> test1 = sscWithTwitter.twitterStream();
+    JavaDStream<Status> test2 = sscWithTwitter.twitterStream(filters);
+    JavaDStream<Status> test3 =
+      sscWithTwitter.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<Status> test4 = sscWithTwitter.twitterStream(auth);
+    JavaDStream<Status> test5 = sscWithTwitter.twitterStream(auth, filters);
+    JavaDStream<Status> test6 =
+      sscWithTwitter.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+
+    // To verify that JavaStreamingContextWithKafka is also StreamingContext
+    JavaDStream<String> socketStream = sscWithTwitter.socketTextStream("localhost", 9999);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties
new file mode 100644
index 0000000..063529a
--- /dev/null
+++ b/external/twitter/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+


[05/13] git commit: Removed unnecessary comments.

Posted by pw...@apache.org.
Removed unnecessary comments.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/3ab297ad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/3ab297ad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/3ab297ad

Branch: refs/heads/master
Commit: 3ab297adaae153cd76c9893ab62e0a815ec21fa4
Parents: 9763084
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Dec 31 00:38:19 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Dec 31 00:38:19 2013 -0800

----------------------------------------------------------------------
 .../spark/streaming/StreamingContext.scala      | 36 +++++---------------
 .../streaming/scheduler/JobGenerator.scala      |  1 -
 .../apache/spark/streaming/JavaAPISuite.java    | 26 --------------
 3 files changed, 8 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ab297ad/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 41898b9..96f57cb 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -25,6 +25,14 @@ import java.io.InputStream
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.UUID
 
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
+import org.apache.hadoop.io.LongWritable
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+import org.apache.hadoop.fs.Path
+
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
@@ -33,14 +41,6 @@ import org.apache.spark.streaming.dstream._
 import org.apache.spark.streaming.receivers._
 import org.apache.spark.streaming.scheduler._
 
-import org.apache.hadoop.io.LongWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
-import org.apache.hadoop.fs.Path
-
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -322,26 +322,6 @@ class StreamingContext private (
     fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
   }
 
-  /*
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
-   *        authorization; this uses the system properties twitter4j.oauth.consumerKey,
-   *        .consumerSecret, .accessToken and .accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Option[Authorization] = None,
-      filters: Seq[String] = Nil,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[Status] = {
-    val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-  */
-
   /**
    * Create an input stream from a queue of RDDs. In each batch,
    * it will process either one or all of the RDDs returned by the queue.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ab297ad/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
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 2734393..1cd0b9b 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
@@ -33,7 +33,6 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
   val ssc = jobScheduler.ssc
   val clockClass = System.getProperty(
     "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
-  logInfo("Using clock class = " + clockClass)
   val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => generateJobs(new Time(longTime)))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3ab297ad/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
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 f4d26c0..6218795 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -1600,30 +1600,4 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa
   public void testRawSocketStream() {
     JavaDStream<String> test = ssc.rawSocketStream("localhost", 12345);
   }
-  /*
-  @Test
-  public void testFileStream() {
-    JavaPairDStream<String, String> foo = ssc.<String, String, SequenceFileInputFormat<String,String>>fileStream("/tmp/foo");
-  }
-
-  @Test
-  public void testTwitterStream() {
-    String[] filters = new String[] { "good", "bad", "ugly" };
-    JavaDStream<Status> test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY());
-  }
-
-  @Test
-  public void testActorStream() {
-    JavaDStream<String> test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
-  }
-
-  @Test
-  public void testZeroMQStream() {
-    JavaDStream<String> test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
-      @Override
-      public Iterable<String> call(byte[][] b) throws Exception {
-        return null;
-      }
-    });
-  } */
 }


[08/13] git commit: Changed JavaStreamingContextWith*** to ***Function in streaming.api.java.*** package. Also fixed packages of Flume and MQTT tests.

Posted by pw...@apache.org.
Changed JavaStreamingContextWith*** to ***Function in streaming.api.java.*** package. Also fixed packages of Flume and MQTT tests.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/d0fd3b9a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/d0fd3b9a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/d0fd3b9a

Branch: refs/heads/master
Commit: d0fd3b9ad238294346eb3465c489eabd41fb2380
Parents: 977bcc3
Author: Tathagata Das <ta...@gmail.com>
Authored: Mon Jan 6 01:47:53 2014 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Mon Jan 6 01:47:53 2014 -0800

----------------------------------------------------------------------
 .../streaming/examples/JavaFlumeEventCount.java |   6 +-
 .../streaming/examples/JavaKafkaWordCount.java  |   6 +-
 .../api/java/flume/FlumeFunctions.scala         |  48 ++++++++
 .../flume/JavaStreamingContextWithFlume.scala   |  48 --------
 .../src/test/java/JavaFlumeStreamSuite.java     |  38 -------
 .../streaming/flume/JavaFlumeStreamSuite.java   |  35 ++++++
 .../api/java/kafka/KafkaFunctions.scala         | 107 ++++++++++++++++++
 .../kafka/JavaStreamingContextWithKafka.scala   | 107 ------------------
 .../streaming/kafka/JavaKafkaStreamSuite.java   |  15 ++-
 .../mqtt/JavaStreamingContextWithMQTT.scala     |  59 ----------
 .../spark/streaming/mqtt/MQTTFunctions.scala    |  43 --------
 .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 -------------------
 .../spark/spark/streaming/mqtt/package.scala    |  24 ----
 .../streaming/api/java/mqtt/MQTTFunctions.scala |  59 ++++++++++
 .../spark/streaming/mqtt/MQTTFunctions.scala    |  43 ++++++++
 .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 +++++++++++++++++++
 .../apache/spark/streaming/mqtt/package.scala   |  24 ++++
 .../streaming/mqtt/JavaMQTTStreamSuite.java     |  10 +-
 .../api/java/twitter/TwitterFunctions.scala     |  99 +++++++++++++++++
 .../JavaStreamingContextWithTwitter.scala       |  99 -----------------
 .../twitter/JavaTwitterStreamSuite.java         |  19 ++--
 .../api/java/zeromq/ZeroMQFunctions.scala       | 102 +++++++++++++++++
 .../zeromq/JavaStreamingContextWithZeroMQ.scala | 102 -----------------
 .../streaming/zeromq/JavaZeroMQStreamSuite.java |  12 +-
 24 files changed, 658 insertions(+), 667 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index e53c4f9..64832a9 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.examples;
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.streaming.*;
 import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume;
+import org.apache.spark.streaming.api.java.flume.FlumeFunctions;
 import org.apache.spark.streaming.flume.SparkFlumeEvent;
 
 /**
@@ -52,8 +52,8 @@ public class JavaFlumeEventCount {
 
     JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
             System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
-    JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc);
-    JavaDStream<SparkFlumeEvent> flumeStream = sscWithFlume.flumeStream("localhost", port);
+    FlumeFunctions flumeFunc = new FlumeFunctions(ssc);
+    JavaDStream<SparkFlumeEvent> flumeStream = flumeFunc.flumeStream("localhost", port);
 
     flumeStream.count();
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index de0420c..207ce8c 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.Duration;
 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.kafka.JavaStreamingContextWithKafka;
+import org.apache.spark.streaming.api.java.kafka.KafkaFunctions;
 import scala.Tuple2;
 
 /**
@@ -64,8 +64,8 @@ public class JavaKafkaWordCount {
       topicMap.put(topic, numThreads);
     }
 
-    JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc);
-    JavaPairDStream<String, String> messages = sscWithKafka.kafkaStream(args[1], args[2], topicMap);
+    KafkaFunctions kafkaFunc = new KafkaFunctions(ssc);
+    JavaPairDStream<String, String> messages = kafkaFunc.kafkaStream(args[1], args[2], topicMap);
 
     JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
new file mode 100644
index 0000000..3347d19
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/api/java/flume/FlumeFunctions.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.api.java.flume
+
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+import org.apache.spark.streaming.flume._
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Flume input streams.
+ */
+class FlumeFunctions(javaStreamingContext: JavaStreamingContext) {
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   */
+  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
+    javaStreamingContext.ssc.flumeStream(hostname, port)
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
+    JavaDStream[SparkFlumeEvent] = {
+    javaStreamingContext.ssc.flumeStream(hostname, port, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
----------------------------------------------------------------------
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
deleted file mode 100644
index 4e66ae3..0000000
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/JavaStreamingContextWithFlume.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.flume
-
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-import org.apache.spark.storage.StorageLevel
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Flume input streams.
- */
-class JavaStreamingContextWithFlume(javaStreamingContext: JavaStreamingContext)
-  extends JavaStreamingContext(javaStreamingContext.ssc) {
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   */
-  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port)
-  }
-
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
-    JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/flume/src/test/java/JavaFlumeStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/flume/src/test/java/JavaFlumeStreamSuite.java b/external/flume/src/test/java/JavaFlumeStreamSuite.java
deleted file mode 100644
index deffc78..0000000
--- a/external/flume/src/test/java/JavaFlumeStreamSuite.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.streaming.LocalJavaStreamingContext;
-import org.apache.spark.streaming.api.java.JavaDStream;
-import org.apache.spark.streaming.flume.JavaStreamingContextWithFlume;
-import org.apache.spark.streaming.flume.SparkFlumeEvent;
-import org.junit.Test;
-
-public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
-  @Test
-  public void testFlumeStream() {
-    JavaStreamingContextWithFlume sscWithFlume = new JavaStreamingContextWithFlume(ssc);
-
-    // tests the API, does not actually test data receiving
-    JavaDStream<SparkFlumeEvent> test1 = sscWithFlume.flumeStream("localhost", 12345);
-    JavaDStream<SparkFlumeEvent> test2 = sscWithFlume.flumeStream("localhost", 12345,
-      StorageLevel.MEMORY_AND_DISK_SER_2());
-
-    // To verify that JavaStreamingContextWithKafka is also StreamingContext
-    JavaDStream<String> socketStream = sscWithFlume.socketTextStream("localhost", 9999);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
new file mode 100644
index 0000000..5930fee
--- /dev/null
+++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
@@ -0,0 +1,35 @@
+package org.apache.spark.streaming.flume;/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.flume.FlumeFunctions;
+import org.apache.spark.streaming.flume.SparkFlumeEvent;
+import org.junit.Test;
+
+public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
+  @Test
+  public void testFlumeStream() {
+    FlumeFunctions flumeFunc = new FlumeFunctions(ssc);
+
+    // tests the API, does not actually test data receiving
+    JavaDStream<SparkFlumeEvent> test1 = flumeFunc.flumeStream("localhost", 12345);
+    JavaDStream<SparkFlumeEvent> test2 = flumeFunc.flumeStream("localhost", 12345,
+      StorageLevel.MEMORY_AND_DISK_SER_2());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
new file mode 100644
index 0000000..491331b
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/api/java/kafka/KafkaFunctions.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.api.java.kafka
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.lang.{Integer => JInt}
+import java.util.{Map => JMap}
+
+import kafka.serializer.Decoder
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
+import org.apache.spark.streaming.kafka._
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Kafka input streams.
+ */
+class KafkaFunctions(javaStreamingContext: JavaStreamingContext) {
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   */
+  def kafkaStream(
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt]
+    ): JavaPairDStream[String, String] = {
+      implicit val cmt: ClassTag[String] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+      javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   *               in its own thread.
+   * @param storageLevel RDD storage level.
+   *
+   */
+  def kafkaStream(
+      zkQuorum: String,
+      groupId: String,
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[String, String] = {
+    implicit val cmt: ClassTag[String] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    javaStreamingContext.ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param keyTypeClass Key type of RDD
+   * @param valueTypeClass value type of RDD
+   * @param keyDecoderClass Type of kafka key decoder
+   * @param valueDecoderClass Type of kafka value decoder
+   * @param kafkaParams Map of kafka configuration paramaters.
+   *                    See: http://kafka.apache.org/configuration.html
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   * @param storageLevel RDD storage level. Defaults to memory-only
+   */
+  def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]](
+      keyTypeClass: Class[K],
+      valueTypeClass: Class[V],
+      keyDecoderClass: Class[U],
+      valueDecoderClass: Class[T],
+      kafkaParams: JMap[String, String],
+      topics: JMap[String, JInt],
+      storageLevel: StorageLevel
+    ): JavaPairDStream[K, V] = {
+    implicit val keyCmt: ClassTag[K] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+    implicit val valueCmt: ClassTag[V] =
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+
+    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+    javaStreamingContext.ssc.kafkaStream[K, V, U, T](
+      kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
deleted file mode 100644
index ab0e8a6..0000000
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/JavaStreamingContextWithKafka.scala
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.kafka
-
-import scala.reflect.ClassTag
-import scala.collection.JavaConversions._
-
-import java.lang.{Integer => JInt}
-import java.util.{Map => JMap}
-
-import kafka.serializer.Decoder
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Kafka input streams.
- */
-class JavaStreamingContextWithKafka(javaStreamingContext: JavaStreamingContext)
-  extends JavaStreamingContext(javaStreamingContext.ssc) {
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: JMap[String, JInt]
-    ): JavaPairDStream[String, String] = {
-      implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-      ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel RDD storage level.
-   *
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: JMap[String, JInt],
-      storageLevel: StorageLevel
-    ): JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param keyTypeClass Key type of RDD
-   * @param valueTypeClass value type of RDD
-   * @param keyDecoderClass Type of kafka key decoder
-   * @param valueDecoderClass Type of kafka value decoder
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   */
-  def kafkaStream[K, V, U <: Decoder[_], T <: Decoder[_]](
-      keyTypeClass: Class[K],
-      valueTypeClass: Class[V],
-      keyDecoderClass: Class[U],
-      valueDecoderClass: Class[T],
-      kafkaParams: JMap[String, String],
-      topics: JMap[String, JInt],
-      storageLevel: StorageLevel
-    ): JavaPairDStream[K, V] = {
-    implicit val keyCmt: ClassTag[K] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
-    implicit val valueCmt: ClassTag[V] =
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
-
-    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
-    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
-
-    ssc.kafkaStream[K, V, U, T](
-      kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
index 66236df..fdea96e 100644
--- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -18,6 +18,8 @@
 package org.apache.spark.streaming.kafka;
 
 import java.util.HashMap;
+
+import org.apache.spark.streaming.api.java.kafka.KafkaFunctions;
 import org.junit.Test;
 import com.google.common.collect.Maps;
 import kafka.serializer.StringDecoder;
@@ -31,21 +33,18 @@ public class JavaKafkaStreamSuite extends LocalJavaStreamingContext {
   public void testKafkaStream() {
 
     HashMap<String, Integer> topics = Maps.newHashMap();
-    JavaStreamingContextWithKafka sscWithKafka = new JavaStreamingContextWithKafka(ssc);
+    KafkaFunctions kafkaFunc = new KafkaFunctions(ssc);
 
     // tests the API, does not actually test data receiving
-    JavaPairDStream<String, String> test1 = sscWithKafka.kafkaStream("localhost:12345", "group", topics);
-    JavaPairDStream<String, String> test2 = sscWithKafka.kafkaStream("localhost:12345", "group", topics,
+    JavaPairDStream<String, String> test1 = kafkaFunc.kafkaStream("localhost:12345", "group", topics);
+    JavaPairDStream<String, String> test2 = kafkaFunc.kafkaStream("localhost:12345", "group", topics,
       StorageLevel.MEMORY_AND_DISK_SER_2());
 
     HashMap<String, String> kafkaParams = Maps.newHashMap();
-    kafkaParams.put("zookeeper.connect","localhost:12345");
+    kafkaParams.put("zookeeper.connect", "localhost:12345");
     kafkaParams.put("group.id","consumer-group");
-    JavaPairDStream<String, String> test3 = sscWithKafka.kafkaStream(
+    JavaPairDStream<String, String> test3 = kafkaFunc.kafkaStream(
       String.class, String.class, StringDecoder.class, StringDecoder.class,
       kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2());
-
-    // To verify that JavaStreamingContextWithKafka is also StreamingContext
-    JavaDStream<String> socketStream = sscWithKafka.socketTextStream("localhost", 9999);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
deleted file mode 100644
index d814da0..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/JavaStreamingContextWithMQTT.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.mqtt
-
-import scala.reflect.ClassTag
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating MQTT input streams.
- */
-class JavaStreamingContextWithMQTT(javaStreamingContext: JavaStreamingContext)
-  extends JavaStreamingContext(javaStreamingContext.ssc) {
-
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String
-    ): JavaDStream[String] = {
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.mqttStream(brokerUrl, topic)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level.
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): JavaDStream[String] = {
-    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.mqttStream(brokerUrl, topic, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
deleted file mode 100644
index 86f4e9c..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTFunctions.scala
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.mqtt
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-
-/**
- * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions.
- */
-class MQTTFunctions(ssc: StreamingContext) {
-  /**
-   * Create an input stream that receives messages pushed by a MQTT publisher.
-   * @param brokerUrl Url of remote MQTT publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
-   */
-  def mqttStream(
-      brokerUrl: String,
-      topic: String,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[String] = {
-    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
-    ssc.registerInputStream(inputStream)
-    inputStream
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
deleted file mode 100644
index c8987a3..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/MQTTInputDStream.scala
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.mqtt
-
-import scala.collection.Map
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
-import scala.reflect.ClassTag
-
-import java.util.Properties
-import java.util.concurrent.Executors
-import java.io.IOException
-
-import org.eclipse.paho.client.mqttv3.MqttCallback
-import org.eclipse.paho.client.mqttv3.MqttClient
-import org.eclipse.paho.client.mqttv3.MqttClientPersistence
-import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
-import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
-import org.eclipse.paho.client.mqttv3.MqttException
-import org.eclipse.paho.client.mqttv3.MqttMessage
-import org.eclipse.paho.client.mqttv3.MqttTopic
-
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.StreamingContext
-import org.apache.spark.streaming.dstream._
-
-/**
- * Input stream that subscribe messages from a Mqtt Broker.
- * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
- * @param brokerUrl Url of remote mqtt publisher
- * @param topic topic name to subscribe to
- * @param storageLevel RDD storage level.
- */
-
-private[streaming] 
-class MQTTInputDStream[T: ClassTag](
-    @transient ssc_ : StreamingContext,
-    brokerUrl: String,
-    topic: String,
-    storageLevel: StorageLevel
-  ) extends NetworkInputDStream[T](ssc_) with Logging {
-  
-  def getReceiver(): NetworkReceiver[T] = {
-    new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]]
-  }
-}
-
-private[streaming] 
-class MQTTReceiver(brokerUrl: String,
-  topic: String,
-  storageLevel: StorageLevel
-  ) extends NetworkReceiver[Any] {
-  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
-  
-  def onStop() {
-    blockGenerator.stop()
-  }
-  
-  def onStart() {
-
-    blockGenerator.start()
-
-    // Set up persistence for messages 
-    var peristance: MqttClientPersistence = new MemoryPersistence()
-
-    // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
-    var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance)
-
-    // Connect to MqttBroker    
-    client.connect()
-
-    // Subscribe to Mqtt topic
-    client.subscribe(topic)
-
-    // Callback automatically triggers as and when new message arrives on specified topic
-    var callback: MqttCallback = new MqttCallback() {
-
-      // Handles Mqtt message 
-      override def messageArrived(arg0: String, arg1: MqttMessage) {
-        blockGenerator += new String(arg1.getPayload())
-      }
-
-      override def deliveryComplete(arg0: IMqttDeliveryToken) {
-      }
-
-      override def connectionLost(arg0: Throwable) {
-        logInfo("Connection lost " + arg0)
-      }
-    }
-
-    // Set up callback for MqttClient
-    client.setCallback(callback)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
deleted file mode 100644
index 28a944f..0000000
--- a/external/mqtt/src/main/scala/org/apache/spark/spark/streaming/mqtt/package.scala
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object mqtt {
-  implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc)
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
new file mode 100644
index 0000000..7212495
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/api/java/mqtt/MQTTFunctions.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.api.java.mqtt
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+import org.apache.spark.streaming.mqtt._
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating MQTT input streams.
+ */
+class MQTTFunctions(javaStreamingContext: JavaStreamingContext) {
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    javaStreamingContext.ssc.mqttStream(brokerUrl, topic)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   * @param storageLevel RDD storage level.
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): JavaDStream[String] = {
+    implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+    javaStreamingContext.ssc.mqttStream(brokerUrl, topic, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
new file mode 100644
index 0000000..86f4e9c
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTFunctions.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+
+/**
+ * Extra MQTT input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversions. Import org.apache.spark.streaming.mqtt._ to use these functions.
+ */
+class MQTTFunctions(ssc: StreamingContext) {
+  /**
+   * Create an input stream that receives messages pushed by a MQTT publisher.
+   * @param brokerUrl Url of remote MQTT publisher
+   * @param topic topic name to subscribe to
+   * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def mqttStream(
+      brokerUrl: String,
+      topic: String,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[String] = {
+    val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
+    ssc.registerInputStream(inputStream)
+    inputStream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
new file mode 100644
index 0000000..c8987a3
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import java.util.Properties
+import java.util.concurrent.Executors
+import java.io.IOException
+
+import org.eclipse.paho.client.mqttv3.MqttCallback
+import org.eclipse.paho.client.mqttv3.MqttClient
+import org.eclipse.paho.client.mqttv3.MqttClientPersistence
+import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
+import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
+import org.eclipse.paho.client.mqttv3.MqttException
+import org.eclipse.paho.client.mqttv3.MqttMessage
+import org.eclipse.paho.client.mqttv3.MqttTopic
+
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
+
+/**
+ * Input stream that subscribe messages from a Mqtt Broker.
+ * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
+ * @param brokerUrl Url of remote mqtt publisher
+ * @param topic topic name to subscribe to
+ * @param storageLevel RDD storage level.
+ */
+
+private[streaming] 
+class MQTTInputDStream[T: ClassTag](
+    @transient ssc_ : StreamingContext,
+    brokerUrl: String,
+    topic: String,
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_) with Logging {
+  
+  def getReceiver(): NetworkReceiver[T] = {
+    new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]]
+  }
+}
+
+private[streaming] 
+class MQTTReceiver(brokerUrl: String,
+  topic: String,
+  storageLevel: StorageLevel
+  ) extends NetworkReceiver[Any] {
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+  
+  def onStop() {
+    blockGenerator.stop()
+  }
+  
+  def onStart() {
+
+    blockGenerator.start()
+
+    // Set up persistence for messages 
+    var peristance: MqttClientPersistence = new MemoryPersistence()
+
+    // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
+    var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance)
+
+    // Connect to MqttBroker    
+    client.connect()
+
+    // Subscribe to Mqtt topic
+    client.subscribe(topic)
+
+    // Callback automatically triggers as and when new message arrives on specified topic
+    var callback: MqttCallback = new MqttCallback() {
+
+      // Handles Mqtt message 
+      override def messageArrived(arg0: String, arg1: MqttMessage) {
+        blockGenerator += new String(arg1.getPayload())
+      }
+
+      override def deliveryComplete(arg0: IMqttDeliveryToken) {
+      }
+
+      override def connectionLost(arg0: Throwable) {
+        logInfo("Connection lost " + arg0)
+      }
+    }
+
+    // Set up callback for MqttClient
+    client.setCallback(callback)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
----------------------------------------------------------------------
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
new file mode 100644
index 0000000..28a944f
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+package object mqtt {
+  implicit def sscToMQTTFunctions(ssc: StreamingContext) = new MQTTFunctions(ssc)
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
index c1f4164..3ddb4d0 100644
--- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
@@ -19,6 +19,7 @@ package org.apache.spark.streaming.mqtt;
 
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.mqtt.MQTTFunctions;
 import org.junit.Test;
 
 import org.apache.spark.streaming.LocalJavaStreamingContext;
@@ -28,14 +29,11 @@ public class JavaMQTTStreamSuite extends LocalJavaStreamingContext {
   public void testMQTTStream() {
     String brokerUrl = "abc";
     String topic = "def";
-    JavaStreamingContextWithMQTT sscWithMQTT = new JavaStreamingContextWithMQTT(ssc);
+    MQTTFunctions mqttFunc = new MQTTFunctions(ssc);
 
     // tests the API, does not actually test data receiving
-    JavaDStream<String> test1 = sscWithMQTT.mqttStream(brokerUrl, topic);
-    JavaDStream<String> test2 = sscWithMQTT.mqttStream(brokerUrl, topic,
+    JavaDStream<String> test1 = mqttFunc.mqttStream(brokerUrl, topic);
+    JavaDStream<String> test2 = mqttFunc.mqttStream(brokerUrl, topic,
       StorageLevel.MEMORY_AND_DISK_SER_2());
-
-    // To verify that JavaStreamingContextWithKafka is also StreamingContext
-    JavaDStream<String> socketStream = sscWithMQTT.socketTextStream("localhost", 9999);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
new file mode 100644
index 0000000..22e297a
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/api/java/twitter/TwitterFunctions.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.api.java.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+import org.apache.spark.streaming.twitter._
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating Twitter input streams.
+ */
+class TwitterFunctions(javaStreamingContext: JavaStreamingContext) {
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   */
+  def twitterStream(): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(None)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def twitterStream(filters: Array[String]): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(None, filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+   * twitter4j.oauth.accessTokenSecret.
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(None, filters, storageLevel)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization
+   */
+  def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(Some(twitterAuth))
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def twitterStream(
+      twitterAuth: Authorization,
+      filters: Array[String]
+    ): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param twitterAuth Twitter4J Authorization object
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      twitterAuth: Authorization,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    javaStreamingContext.ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
deleted file mode 100644
index 0250364..0000000
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/JavaStreamingContextWithTwitter.scala
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.twitter
-
-import twitter4j.Status
-import twitter4j.auth.Authorization
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating Twitter input streams.
- */
-class JavaStreamingContextWithTwitter(javaStreamingContext: JavaStreamingContext)
-  extends JavaStreamingContext(javaStreamingContext.ssc) {
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   */
-  def twitterStream(): JavaDStream[Status] = {
-    ssc.twitterStream(None)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(filters: Array[String]): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
-   * twitter4j.oauth.accessTokenSecret.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(filters: Array[String], storageLevel: StorageLevel): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   */
-  def twitterStream(twitterAuth: Authorization): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth))
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization object
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
index 34e4fbd..4564d6c 100644
--- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
@@ -18,6 +18,8 @@
 package org.apache.spark.streaming.twitter;
 
 import java.util.Arrays;
+
+import org.apache.spark.streaming.api.java.twitter.TwitterFunctions;
 import org.junit.Test;
 
 import twitter4j.Status;
@@ -31,21 +33,18 @@ import org.apache.spark.streaming.api.java.JavaDStream;
 public class JavaTwitterStreamSuite extends LocalJavaStreamingContext {
   @Test
   public void testTwitterStream() {
-    JavaStreamingContextWithTwitter sscWithTwitter = new JavaStreamingContextWithTwitter(ssc);
+    TwitterFunctions twitterFunc = new TwitterFunctions(ssc);
     String[] filters = (String[])Arrays.<String>asList("filter1", "filter2").toArray();
     Authorization auth = NullAuthorization.getInstance();
 
     // tests the API, does not actually test data receiving
-    JavaDStream<Status> test1 = sscWithTwitter.twitterStream();
-    JavaDStream<Status> test2 = sscWithTwitter.twitterStream(filters);
+    JavaDStream<Status> test1 = twitterFunc.twitterStream();
+    JavaDStream<Status> test2 = twitterFunc.twitterStream(filters);
     JavaDStream<Status> test3 =
-      sscWithTwitter.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2());
-    JavaDStream<Status> test4 = sscWithTwitter.twitterStream(auth);
-    JavaDStream<Status> test5 = sscWithTwitter.twitterStream(auth, filters);
+      twitterFunc.twitterStream(filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<Status> test4 = twitterFunc.twitterStream(auth);
+    JavaDStream<Status> test5 = twitterFunc.twitterStream(auth, filters);
     JavaDStream<Status> test6 =
-      sscWithTwitter.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
-
-    // To verify that JavaStreamingContextWithKafka is also StreamingContext
-    JavaDStream<String> socketStream = sscWithTwitter.socketTextStream("localhost", 9999);
+      twitterFunc.twitterStream(auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
new file mode 100644
index 0000000..a9bbce7
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/api/java/zeromq/ZeroMQFunctions.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.api.java.zeromq
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import akka.actor.SupervisorStrategy
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+import org.apache.spark.streaming.zeromq._
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating ZeroMQ input streams.
+ */
+class ZeroMQFunctions(javaStreamingContext: JavaStreamingContext) {
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote ZeroMQ publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel RDD storage level.
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    javaStreamingContext.ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
deleted file mode 100644
index dc5d1f0..0000000
--- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.zeromq
-
-import scala.reflect.ClassTag
-import scala.collection.JavaConversions._
-
-import akka.actor.SupervisorStrategy
-import akka.util.ByteString
-import akka.zeromq.Subscribe
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.api.java.function.{Function => JFunction}
-import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
-
-/**
- * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
- * functions for creating ZeroMQ input streams.
- */
-class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext)
-  extends JavaStreamingContext(javaStreamingContext.ssc) {
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote ZeroMQ publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel,
-      supervisorStrategy: SupervisorStrategy
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level.
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   */
-  def zeroMQStream[T](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d0fd3b9a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
index 96af7d7..b020ae4 100644
--- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
@@ -17,6 +17,7 @@
 
 package org.apache.spark.streaming.zeromq;
 
+import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions;
 import org.junit.Test;
 
 import akka.actor.SupervisorStrategy;
@@ -32,7 +33,7 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
 
   @Test // tests the API, does not actually test data receiving
   public void testZeroMQStream() {
-    JavaStreamingContextWithZeroMQ sscWithZeroMQ = new JavaStreamingContextWithZeroMQ(ssc);
+    ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc);
     String publishUrl = "abc";
     Subscribe subscribe = new Subscribe((ByteString)null);
     Function<byte[][], Iterable<String>> bytesToObjects = new Function<byte[][], Iterable<String>>() {
@@ -42,14 +43,11 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
       }
     };
 
-    JavaDStream<String> test1 = sscWithZeroMQ.<String>zeroMQStream(
+    JavaDStream<String> test1 = zeromqFunc.<String>zeroMQStream(
       publishUrl, subscribe, bytesToObjects);
-    JavaDStream<String> test2 = sscWithZeroMQ.<String>zeroMQStream(
+    JavaDStream<String> test2 = zeromqFunc.<String>zeroMQStream(
       publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
-    JavaDStream<String> test3 = sscWithZeroMQ.<String>zeroMQStream(
+    JavaDStream<String> test3 = zeromqFunc.<String>zeroMQStream(
       publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy());
-
-    // To verify that JavaStreamingContextWithKafka is also StreamingContext
-    JavaDStream<String> socketStream = sscWithZeroMQ.socketTextStream("localhost", 9999);
   }
 }


[13/13] git commit: Merge pull request #313 from tdas/project-refactor

Posted by pw...@apache.org.
Merge pull request #313 from tdas/project-refactor

Refactored the streaming project to separate external libraries like Twitter, Kafka, Flume, etc.

At a high level, these are the following changes.

1. All the external code was put in `SPARK_HOME/external/` as separate SBT projects and Maven modules. Their artifact names are `spark-streaming-twitter`, `spark-streaming-kafka`, etc. Both SparkBuild.scala and pom.xml files have been updated. References to external libraries and repositories have been removed from the settings of root and streaming projects/modules.

2. To avail the external functionality (say, creating a Twitter stream), the developer has to `import org.apache.spark.streaming.twitter._` . For Scala API, the developer has to call `TwitterUtils.createStream(streamingContext, ...)`. For the Java API, the developer has to call `TwitterUtils.createStream(javaStreamingContext, ...)`.

3.  Each external project has its own scala and java unit tests. Note the unit tests of each external library use classes of the streaming unit tests (`TestSuiteBase`, `LocalJavaStreamingContext`, etc.). To enable this code sharing among test classes, `dependsOn(streaming % "compile->compile,test->test")` was used in the SparkBuild.scala . In the streaming/pom.xml, an additional `maven-jar-plugin` was necessary to capture this dependency (see comment inside the pom.xml for more information).

4. Jars of the external projects have been added to examples project but not to the assembly project.

5. In some files, imports have been rearrange to conform to the Spark coding guidelines.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c0f0155e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c0f0155e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c0f0155e

Branch: refs/heads/master
Commit: c0f0155eca6405d0768a476f0be00594e478fce0
Parents: f5f12dc 8f02f1c
Author: Patrick Wendell <pw...@gmail.com>
Authored: Tue Jan 7 22:21:52 2014 -0800
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Tue Jan 7 22:21:52 2014 -0800

----------------------------------------------------------------------
 bin/run-example                                 |   7 +-
 examples/pom.xml                                |  42 ++--
 .../streaming/examples/JavaFlumeEventCount.java |  10 +-
 .../streaming/examples/JavaKafkaWordCount.java  |   7 +-
 .../streaming/examples/FlumeEventCount.scala    |   3 +-
 .../streaming/examples/KafkaWordCount.scala     |   3 +-
 .../streaming/examples/MQTTWordCount.scala      |  12 +-
 .../streaming/examples/TwitterAlgebirdCMS.scala |   6 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |   4 +-
 .../streaming/examples/TwitterPopularTags.scala |   3 +-
 .../streaming/examples/ZeroMQWordCount.scala    |   9 +-
 external/flume/pom.xml                          |  93 +++++++
 .../streaming/flume/FlumeInputDStream.scala     | 155 ++++++++++++
 .../spark/streaming/flume/FlumeUtils.scala      |  70 ++++++
 .../streaming/flume/JavaFlumeStreamSuite.java   |  34 +++
 .../flume/src/test/resources/log4j.properties   |  29 +++
 .../streaming/flume/FlumeStreamSuite.scala      |  86 +++++++
 external/kafka/pom.xml                          |  97 ++++++++
 .../streaming/kafka/KafkaInputDStream.scala     | 154 ++++++++++++
 .../spark/streaming/kafka/KafkaUtils.scala      | 153 ++++++++++++
 .../streaming/kafka/JavaKafkaStreamSuite.java   |  45 ++++
 .../kafka/src/test/resources/log4j.properties   |  29 +++
 .../streaming/kafka/KafkaStreamSuite.scala      |  39 +++
 external/mqtt/pom.xml                           | 108 +++++++++
 .../spark/streaming/mqtt/MQTTInputDStream.scala | 110 +++++++++
 .../apache/spark/streaming/mqtt/MQTTUtils.scala |  75 ++++++
 .../streaming/mqtt/JavaMQTTStreamSuite.java     |  37 +++
 .../mqtt/src/test/resources/log4j.properties    |  29 +++
 .../spark/streaming/mqtt/MQTTStreamSuite.scala  |  36 +++
 external/twitter/pom.xml                        |  89 +++++++
 .../streaming/twitter/TwitterInputDStream.scala | 100 ++++++++
 .../spark/streaming/twitter/TwitterUtils.scala  | 126 ++++++++++
 .../twitter/JavaTwitterStreamSuite.java         |  46 ++++
 .../twitter/src/test/resources/log4j.properties |  29 +++
 .../streaming/twitter/TwitterStreamSuite.scala  |  43 ++++
 external/zeromq/pom.xml                         |  89 +++++++
 .../spark/streaming/zeromq/ZeroMQReceiver.scala |  54 +++++
 .../spark/streaming/zeromq/ZeroMQUtils.scala    | 126 ++++++++++
 .../streaming/zeromq/JavaZeroMQStreamSuite.java |  50 ++++
 .../zeromq/src/test/resources/log4j.properties  |  29 +++
 .../streaming/zeromq/ZeroMQStreamSuite.scala    |  44 ++++
 pom.xml                                         |  29 +--
 project/SparkBuild.scala                        |  93 +++++--
 streaming/pom.xml                               |  83 +++----
 .../spark/streaming/StreamingContext.scala      | 145 +----------
 .../api/java/JavaStreamingContext.scala         | 243 +------------------
 .../streaming/dstream/FlumeInputDStream.scala   | 154 ------------
 .../streaming/dstream/KafkaInputDStream.scala   | 153 ------------
 .../streaming/dstream/MQTTInputDStream.scala    | 110 ---------
 .../streaming/dstream/TwitterInputDStream.scala |  99 --------
 .../streaming/receivers/ZeroMQReceiver.scala    |  53 ----
 .../scheduler/NetworkInputTracker.scala         |   4 +-
 .../apache/spark/streaming/JavaAPISuite.java    | 104 +-------
 .../streaming/LocalJavaStreamingContext.java    |  46 ++++
 .../spark/streaming/InputStreamsSuite.scala     |  82 +------
 .../apache/spark/streaming/TestSuiteBase.scala  |  20 +-
 56 files changed, 2448 insertions(+), 1280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/bin/run-example
----------------------------------------------------------------------
diff --cc bin/run-example
index 6c5d4a6,a94913d..2e9d514
--- a/bin/run-example
+++ b/bin/run-example
@@@ -45,17 -45,11 +45,11 @@@ f
  EXAMPLES_DIR="$FWDIR"/examples
  SPARK_EXAMPLES_JAR=""
  if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
-   # Use the JAR from the SBT build
    export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
  fi
- if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
-   # Use the JAR from the Maven build
-   # TODO: this also needs to become an assembly!
-   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
- fi
  if [[ -z $SPARK_EXAMPLES_JAR ]]; then
    echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
 -  echo "You need to build Spark with sbt assembly before running this program" >&2
 +  echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
    exit 1
  fi
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/examples/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c0f0155e/project/SparkBuild.scala
----------------------------------------------------------------------


[09/13] git commit: Merge remote-tracking branch 'apache/master' into project-refactor

Posted by pw...@apache.org.
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


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/3b4c4c7f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/3b4c4c7f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/3b4c4c7f

Branch: refs/heads/master
Commit: 3b4c4c7f4d0d6e45a1acb0baf0d9416a8997b686
Parents: d0fd3b9 a2e7e04
Author: Tathagata Das <ta...@gmail.com>
Authored: Mon Jan 6 03:05:52 2014 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Mon Jan 6 03:05:52 2014 -0800

----------------------------------------------------------------------
 .gitignore                                      |    2 +
 README.md                                       |   28 +-
 assembly/lib/PY4J_LICENSE.txt                   |   27 -
 assembly/lib/PY4J_VERSION.txt                   |    1 -
 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar  |  Bin 103286 -> 0 bytes
 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom  |    9 -
 .../net/sf/py4j/py4j/maven-metadata-local.xml   |   12 -
 assembly/pom.xml                                |   14 +-
 assembly/src/main/assembly/assembly.xml         |   11 +-
 bin/compute-classpath.cmd                       |    2 +-
 bin/compute-classpath.sh                        |    2 +-
 bin/pyspark                                     |   70 ++
 bin/pyspark.cmd                                 |   23 +
 bin/pyspark2.cmd                                |   55 +
 bin/run-example                                 |   91 ++
 bin/run-example.cmd                             |   23 +
 bin/run-example2.cmd                            |   61 ++
 bin/slaves.sh                                   |   91 --
 bin/spark-class                                 |  154 +++
 bin/spark-class.cmd                             |   23 +
 bin/spark-class2.cmd                            |   85 ++
 bin/spark-config.sh                             |   36 -
 bin/spark-daemon.sh                             |  183 ----
 bin/spark-daemons.sh                            |   35 -
 bin/spark-shell                                 |  102 ++
 bin/spark-shell.cmd                             |   23 +
 bin/start-all.sh                                |   34 -
 bin/start-master.sh                             |   52 -
 bin/start-slave.sh                              |   35 -
 bin/start-slaves.sh                             |   48 -
 bin/stop-all.sh                                 |   32 -
 bin/stop-master.sh                              |   27 -
 bin/stop-slaves.sh                              |   35 -
 core/pom.xml                                    |  422 ++++----
 .../apache/spark/network/netty/FileClient.java  |   32 +-
 .../netty/FileClientChannelInitializer.java     |    6 +-
 .../spark/network/netty/FileClientHandler.java  |   12 +-
 .../apache/spark/network/netty/FileServer.java  |   29 +-
 .../netty/FileServerChannelInitializer.java     |    3 +-
 .../spark/network/netty/FileServerHandler.java  |   18 +-
 .../org/apache/spark/default-log4j.properties   |    8 +
 .../scala/org/apache/spark/Accumulators.scala   |    8 +-
 .../scala/org/apache/spark/HttpServer.scala     |    1 +
 .../main/scala/org/apache/spark/Logging.scala   |   41 +-
 .../org/apache/spark/MapOutputTracker.scala     |   11 +-
 .../scala/org/apache/spark/Partitioner.scala    |    4 +-
 .../main/scala/org/apache/spark/SparkConf.scala |  189 ++++
 .../scala/org/apache/spark/SparkContext.scala   |  297 ++++--
 .../main/scala/org/apache/spark/SparkEnv.scala  |   54 +-
 .../org/apache/spark/api/java/JavaPairRDD.scala |   36 +
 .../org/apache/spark/api/java/JavaRDDLike.scala |   11 +
 .../spark/api/java/JavaSparkContext.scala       |   56 +-
 .../org/apache/spark/api/python/PythonRDD.scala |    4 +-
 .../org/apache/spark/broadcast/Broadcast.scala  |    8 +-
 .../spark/broadcast/BroadcastFactory.scala      |    4 +-
 .../apache/spark/broadcast/HttpBroadcast.scala  |   43 +-
 .../spark/broadcast/TorrentBroadcast.scala      |   45 +-
 .../spark/deploy/FaultToleranceTest.scala       |    4 +-
 .../apache/spark/deploy/LocalSparkCluster.scala |    7 +-
 .../apache/spark/deploy/SparkHadoopUtil.scala   |   14 +-
 .../org/apache/spark/deploy/client/Client.scala |   21 +-
 .../apache/spark/deploy/client/TestClient.scala |   10 +-
 .../org/apache/spark/deploy/master/Master.scala |   41 +-
 .../spark/deploy/master/MasterArguments.scala   |   11 +-
 .../deploy/master/SparkZooKeeperSession.scala   |    7 +-
 .../master/ZooKeeperLeaderElectionAgent.scala   |    9 +-
 .../master/ZooKeeperPersistenceEngine.scala     |    8 +-
 .../spark/deploy/master/ui/MasterWebUI.scala    |    2 +-
 .../org/apache/spark/deploy/worker/Worker.scala |   34 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |    6 +-
 .../executor/CoarseGrainedExecutorBackend.scala |    6 +-
 .../org/apache/spark/executor/Executor.scala    |   47 +-
 .../org/apache/spark/io/CompressionCodec.scala  |   19 +-
 .../apache/spark/metrics/MetricsConfig.scala    |    1 -
 .../apache/spark/metrics/MetricsSystem.scala    |   11 +-
 .../spark/network/ConnectionManager.scala       |   22 +-
 .../org/apache/spark/network/ReceiverTest.scala |   12 +-
 .../org/apache/spark/network/SenderTest.scala   |   16 +-
 .../spark/network/netty/ShuffleCopier.scala     |   10 +-
 .../org/apache/spark/rdd/CheckpointRDD.scala    |   37 +-
 .../org/apache/spark/rdd/CoGroupedRDD.scala     |    2 +-
 .../org/apache/spark/rdd/PairRDDFunctions.scala |   42 +
 .../spark/rdd/PartitionerAwareUnionRDD.scala    |  110 ++
 .../main/scala/org/apache/spark/rdd/RDD.scala   |   19 +-
 .../apache/spark/rdd/RDDCheckpointData.scala    |   17 +-
 .../org/apache/spark/rdd/ShuffledRDD.scala      |    2 +-
 .../org/apache/spark/rdd/SubtractedRDD.scala    |    2 +-
 .../apache/spark/scheduler/DAGScheduler.scala   |    3 +-
 .../spark/scheduler/InputFormatInfo.scala       |   14 +-
 .../org/apache/spark/scheduler/ResultTask.scala |    4 +-
 .../spark/scheduler/SchedulableBuilder.scala    |    6 +-
 .../spark/scheduler/SchedulerBackend.scala      |    3 -
 .../apache/spark/scheduler/ShuffleMapTask.scala |    6 +-
 .../spark/scheduler/TaskResultGetter.scala      |    3 +-
 .../spark/scheduler/TaskSchedulerImpl.scala     |   25 +-
 .../apache/spark/scheduler/TaskSetManager.scala |   23 +-
 .../cluster/CoarseGrainedSchedulerBackend.scala |   20 +-
 .../cluster/SimrSchedulerBackend.scala          |    4 +-
 .../cluster/SparkDeploySchedulerBackend.scala   |    8 +-
 .../mesos/CoarseMesosSchedulerBackend.scala     |   18 +-
 .../cluster/mesos/MesosSchedulerBackend.scala   |   12 +-
 .../spark/scheduler/local/LocalBackend.scala    |    3 +-
 .../spark/serializer/JavaSerializer.scala       |    3 +-
 .../spark/serializer/KryoSerializer.scala       |   14 +-
 .../apache/spark/serializer/Serializer.scala    |    3 +
 .../spark/serializer/SerializerManager.scala    |   23 +-
 .../spark/storage/BlockFetcherIterator.scala    |    4 +-
 .../org/apache/spark/storage/BlockManager.scala |   58 +-
 .../spark/storage/BlockManagerMaster.scala      |   11 +-
 .../spark/storage/BlockManagerMasterActor.scala |   16 +-
 .../spark/storage/BlockManagerWorker.scala      |    3 -
 .../spark/storage/BlockMessageArray.scala       |    2 -
 .../spark/storage/BlockObjectWriter.scala       |    5 +-
 .../apache/spark/storage/DiskBlockManager.scala |    2 +-
 .../spark/storage/ShuffleBlockManager.scala     |   10 +-
 .../spark/storage/StoragePerfTester.scala       |    2 +-
 .../apache/spark/storage/ThreadingTest.scala    |    8 +-
 .../scala/org/apache/spark/ui/SparkUI.scala     |    4 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala   |   21 +-
 .../org/apache/spark/ui/env/EnvironmentUI.scala |   15 +-
 .../spark/ui/jobs/JobProgressListener.scala     |    4 +-
 .../scala/org/apache/spark/util/AkkaUtils.scala |   39 +-
 .../org/apache/spark/util/MetadataCleaner.scala |   35 +-
 .../spark/util/SerializableHyperLogLog.scala    |   50 +
 .../org/apache/spark/util/SizeEstimator.scala   |   14 +-
 .../scala/org/apache/spark/util/Utils.scala     |   25 +-
 core/src/test/resources/spark.conf              |    8 +
 .../test/resources/uncommons-maths-1.2.2.jar    |  Bin 49019 -> 0 bytes
 .../org/apache/spark/CheckpointSuite.scala      |  363 ++++---
 .../scala/org/apache/spark/DriverSuite.scala    |    8 +-
 .../org/apache/spark/FileServerSuite.scala      |  108 +-
 .../scala/org/apache/spark/JavaAPISuite.java    |   36 +-
 .../apache/spark/MapOutputTrackerSuite.scala    |   16 +-
 .../org/apache/spark/SharedSparkContext.scala   |    4 +-
 .../scala/org/apache/spark/SparkConfSuite.scala |  110 ++
 .../deploy/worker/ExecutorRunnerTest.scala      |    4 +-
 .../apache/spark/io/CompressionCodecSuite.scala |    8 +-
 .../spark/metrics/MetricsSystemSuite.scala      |    8 +-
 .../spark/rdd/PairRDDFunctionsSuite.scala       |   34 +
 .../scala/org/apache/spark/rdd/RDDSuite.scala   |   40 +
 .../spark/scheduler/ClusterSchedulerSuite.scala |    2 +-
 .../spark/scheduler/DAGSchedulerSuite.scala     |   23 +-
 .../apache/spark/scheduler/JobLoggerSuite.scala |    2 +-
 .../spark/scheduler/TaskResultGetterSuite.scala |    6 +-
 .../spark/scheduler/TaskSetManagerSuite.scala   |    4 +-
 .../spark/serializer/KryoSerializerSuite.scala  |   33 +-
 .../spark/storage/BlockManagerSuite.scala       |   97 +-
 .../spark/storage/DiskBlockManagerSuite.scala   |   18 +-
 .../apache/spark/util/SizeEstimatorSuite.scala  |    2 +-
 data/kmeans_data.txt                            |    6 +
 data/lr_data.txt                                | 1000 ++++++++++++++++++
 data/pagerank_data.txt                          |    6 +
 docs/README.md                                  |    4 +-
 docs/_config.yml                                |    2 +-
 docs/_plugins/copy_api_dirs.rb                  |    4 +-
 docs/api.md                                     |    2 +-
 docs/bagel-programming-guide.md                 |    4 +-
 docs/building-with-maven.md                     |   14 +-
 docs/configuration.md                           |   71 +-
 docs/css/bootstrap.min.css                      |    2 +-
 docs/hadoop-third-party-distributions.md        |    2 +-
 docs/index.md                                   |   16 +-
 docs/java-programming-guide.md                  |    4 +-
 docs/job-scheduling.md                          |   21 +-
 docs/mllib-guide.md                             |    2 +-
 docs/monitoring.md                              |    3 +-
 docs/python-programming-guide.md                |   45 +-
 docs/quick-start.md                             |   62 +-
 docs/running-on-mesos.md                        |   19 +-
 docs/running-on-yarn.md                         |   17 +-
 docs/scala-programming-guide.md                 |   20 +-
 docs/spark-debugger.md                          |    2 +-
 docs/spark-standalone.md                        |   35 +-
 docs/streaming-programming-guide.md             |    8 +-
 docs/tuning.md                                  |   21 +-
 ec2/spark_ec2.py                                |    2 +-
 .../org/apache/spark/examples/JavaHdfsLR.java   |    2 +-
 .../org/apache/spark/examples/JavaKMeans.java   |    2 +-
 .../org/apache/spark/examples/JavaLogQuery.java |    2 +-
 .../org/apache/spark/examples/JavaPageRank.java |    3 +-
 .../org/apache/spark/examples/JavaSparkPi.java  |    2 +-
 .../java/org/apache/spark/examples/JavaTC.java  |    2 +-
 .../apache/spark/examples/JavaWordCount.java    |    2 +-
 .../apache/spark/mllib/examples/JavaALS.java    |    2 +-
 .../apache/spark/mllib/examples/JavaKMeans.java |    2 +-
 .../org/apache/spark/mllib/examples/JavaLR.java |    2 +-
 .../streaming/examples/JavaFlumeEventCount.java |    3 +-
 .../streaming/examples/JavaKafkaWordCount.java  |    5 +-
 .../examples/JavaNetworkWordCount.java          |    3 +-
 .../streaming/examples/JavaQueueStream.java     |    2 +-
 .../apache/spark/examples/BroadcastTest.scala   |    2 +-
 .../spark/examples/ExceptionHandlingTest.scala  |    2 +-
 .../org/apache/spark/examples/GroupByTest.scala |    2 +-
 .../org/apache/spark/examples/HBaseTest.scala   |    2 +-
 .../org/apache/spark/examples/HdfsTest.scala    |    2 +-
 .../org/apache/spark/examples/LogQuery.scala    |    2 +-
 .../spark/examples/MultiBroadcastTest.scala     |    2 +-
 .../examples/SimpleSkewedGroupByTest.scala      |    2 +-
 .../spark/examples/SkewedGroupByTest.scala      |    2 +-
 .../org/apache/spark/examples/SparkALS.scala    |    2 +-
 .../org/apache/spark/examples/SparkHdfsLR.scala |    2 +-
 .../org/apache/spark/examples/SparkKMeans.scala |    2 +-
 .../org/apache/spark/examples/SparkLR.scala     |    2 +-
 .../apache/spark/examples/SparkPageRank.scala   |    2 +-
 .../org/apache/spark/examples/SparkPi.scala     |    2 +-
 .../org/apache/spark/examples/SparkTC.scala     |    2 +-
 .../examples/bagel/WikipediaPageRank.scala      |   10 +-
 .../bagel/WikipediaPageRankStandalone.scala     |    8 +-
 .../streaming/examples/ActorWordCount.scala     |    9 +-
 .../streaming/examples/FlumeEventCount.scala    |    2 +-
 .../streaming/examples/HdfsWordCount.scala      |    4 +-
 .../streaming/examples/KafkaWordCount.scala     |    4 +-
 .../streaming/examples/MQTTWordCount.scala      |    6 +-
 .../streaming/examples/NetworkWordCount.scala   |    4 +-
 .../spark/streaming/examples/QueueStream.scala  |    2 +-
 .../streaming/examples/RawNetworkGrep.scala     |    2 +-
 .../examples/StatefulNetworkWordCount.scala     |    4 +-
 .../streaming/examples/TwitterAlgebirdCMS.scala |    2 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |    2 +-
 .../streaming/examples/TwitterPopularTags.scala |    2 +-
 .../streaming/examples/ZeroMQWordCount.scala    |    6 +-
 .../clickstream/PageViewGenerator.scala         |    4 +-
 .../examples/clickstream/PageViewStream.scala   |    6 +-
 .../streaming/flume/FlumeStreamSuite.scala      |    2 +-
 kmeans_data.txt                                 |    6 -
 lr_data.txt                                     | 1000 ------------------
 make-distribution.sh                            |   24 +-
 .../spark/mllib/classification/NaiveBayes.scala |  119 +++
 .../apache/spark/mllib/recommendation/ALS.scala |   13 +-
 .../mllib/classification/NaiveBayesSuite.scala  |  108 ++
 new-yarn/pom.xml                                |  161 ---
 .../spark/deploy/yarn/ApplicationMaster.scala   |  446 --------
 .../yarn/ApplicationMasterArguments.scala       |   94 --
 .../org/apache/spark/deploy/yarn/Client.scala   |  521 ---------
 .../spark/deploy/yarn/ClientArguments.scala     |  149 ---
 .../yarn/ClientDistributedCacheManager.scala    |  228 ----
 .../spark/deploy/yarn/WorkerLauncher.scala      |  222 ----
 .../spark/deploy/yarn/WorkerRunnable.scala      |  209 ----
 .../deploy/yarn/YarnAllocationHandler.scala     |  687 ------------
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |   43 -
 .../cluster/YarnClientClusterScheduler.scala    |   48 -
 .../cluster/YarnClientSchedulerBackend.scala    |  110 --
 .../cluster/YarnClusterScheduler.scala          |   56 -
 .../ClientDistributedCacheManagerSuite.scala    |  220 ----
 pagerank_data.txt                               |    6 -
 pom.xml                                         |   66 +-
 project/SparkBuild.scala                        |   54 +-
 pyspark                                         |   70 --
 pyspark.cmd                                     |   23 -
 pyspark2.cmd                                    |   55 -
 python/epydoc.conf                              |    2 +-
 python/lib/py4j-0.8.1-src.zip                   |  Bin 0 -> 37662 bytes
 python/lib/py4j0.7.egg                          |  Bin 191756 -> 0 bytes
 python/pyspark/__init__.py                      |   34 +-
 python/pyspark/broadcast.py                     |   11 +
 python/pyspark/conf.py                          |  171 +++
 python/pyspark/context.py                       |   68 +-
 python/pyspark/java_gateway.py                  |    3 +-
 python/pyspark/rdd.py                           |   66 +-
 python/pyspark/shell.py                         |    2 +-
 python/pyspark/tests.py                         |    4 +-
 python/run-tests                                |    3 +-
 repl-bin/src/deb/bin/run                        |    3 +-
 repl/pom.xml                                    |    1 -
 .../org/apache/spark/repl/SparkILoop.scala      |   19 +-
 .../org/apache/spark/repl/SparkIMain.scala      |    7 +-
 run-example                                     |   91 --
 run-example.cmd                                 |   23 -
 run-example2.cmd                                |   61 --
 sbin/slaves.sh                                  |   91 ++
 sbin/spark-config.sh                            |   36 +
 sbin/spark-daemon.sh                            |  183 ++++
 sbin/spark-daemons.sh                           |   35 +
 sbin/spark-executor                             |   23 +
 sbin/start-all.sh                               |   34 +
 sbin/start-master.sh                            |   52 +
 sbin/start-slave.sh                             |   35 +
 sbin/start-slaves.sh                            |   48 +
 sbin/stop-all.sh                                |   32 +
 sbin/stop-master.sh                             |   27 +
 sbin/stop-slaves.sh                             |   35 +
 sbt/sbt                                         |   43 -
 sbt/sbt-launch-0.11.3-2.jar                     |  Bin 1096763 -> 0 bytes
 sbt/sbt.cmd                                     |   25 -
 spark-class                                     |  149 ---
 spark-class.cmd                                 |   23 -
 spark-class2.cmd                                |   85 --
 spark-executor                                  |   22 -
 spark-shell                                     |  102 --
 spark-shell.cmd                                 |   22 -
 .../org/apache/spark/streaming/Checkpoint.scala |   66 +-
 .../org/apache/spark/streaming/DStream.scala    |    4 +-
 .../apache/spark/streaming/DStreamGraph.scala   |    1 -
 .../spark/streaming/PairDStreamFunctions.scala  |   13 +-
 .../spark/streaming/StreamingContext.scala      |   89 +-
 .../streaming/api/java/JavaPairDStream.scala    |   18 +-
 .../api/java/JavaStreamingContext.scala         |   38 +-
 .../streaming/dstream/FileInputDStream.scala    |  153 +--
 .../streaming/dstream/NetworkInputDStream.scala |    8 +-
 .../streaming/dstream/ShuffledDStream.scala     |    9 +-
 .../streaming/dstream/WindowedDStream.scala     |   16 +-
 .../streaming/scheduler/JobGenerator.scala      |   71 +-
 .../streaming/scheduler/JobScheduler.scala      |    6 +-
 .../streaming/util/MasterFailureTest.scala      |    3 -
 .../spark/streaming/util/RawTextSender.scala    |    4 +-
 .../apache/spark/streaming/JavaAPISuite.java    |    4 +-
 .../spark/streaming/BasicOperationsSuite.scala  |    8 +-
 .../spark/streaming/CheckpointSuite.scala       |   59 +-
 .../spark/streaming/InputStreamsSuite.scala     |   15 +-
 .../apache/spark/streaming/TestSuiteBase.scala  |   21 +-
 .../spark/streaming/WindowOperationsSuite.scala |    5 +-
 yarn/README.md                                  |   12 +
 yarn/alpha/pom.xml                              |   32 +
 .../spark/deploy/yarn/ApplicationMaster.scala   |  464 ++++++++
 .../org/apache/spark/deploy/yarn/Client.scala   |  509 +++++++++
 .../spark/deploy/yarn/WorkerLauncher.scala      |  250 +++++
 .../spark/deploy/yarn/WorkerRunnable.scala      |  236 +++++
 .../deploy/yarn/YarnAllocationHandler.scala     |  680 ++++++++++++
 .../yarn/ApplicationMasterArguments.scala       |   94 ++
 .../spark/deploy/yarn/ClientArguments.scala     |  150 +++
 .../yarn/ClientDistributedCacheManager.scala    |  228 ++++
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |   43 +
 .../cluster/YarnClientClusterScheduler.scala    |   48 +
 .../cluster/YarnClientSchedulerBackend.scala    |  110 ++
 .../cluster/YarnClusterScheduler.scala          |   56 +
 .../ClientDistributedCacheManagerSuite.scala    |  220 ++++
 yarn/pom.xml                                    |   84 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |  477 ---------
 .../yarn/ApplicationMasterArguments.scala       |   94 --
 .../org/apache/spark/deploy/yarn/Client.scala   |  503 ---------
 .../spark/deploy/yarn/ClientArguments.scala     |  146 ---
 .../yarn/ClientDistributedCacheManager.scala    |  228 ----
 .../spark/deploy/yarn/WorkerLauncher.scala      |  243 -----
 .../spark/deploy/yarn/WorkerRunnable.scala      |  235 ----
 .../deploy/yarn/YarnAllocationHandler.scala     |  673 ------------
 .../spark/deploy/yarn/YarnSparkHadoopUtil.scala |   43 -
 .../cluster/YarnClientClusterScheduler.scala    |   48 -
 .../cluster/YarnClientSchedulerBackend.scala    |  110 --
 .../cluster/YarnClusterScheduler.scala          |   59 --
 .../ClientDistributedCacheManagerSuite.scala    |  220 ----
 yarn/stable/pom.xml                             |   32 +
 .../spark/deploy/yarn/ApplicationMaster.scala   |  432 ++++++++
 .../org/apache/spark/deploy/yarn/Client.scala   |  525 +++++++++
 .../spark/deploy/yarn/WorkerLauncher.scala      |  230 ++++
 .../spark/deploy/yarn/WorkerRunnable.scala      |  210 ++++
 .../deploy/yarn/YarnAllocationHandler.scala     |  695 ++++++++++++
 346 files changed, 11176 insertions(+), 10576 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------
diff --cc examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 64832a9,64ac724..83900a1
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@@ -50,10 -49,11 +50,11 @@@ public class JavaFlumeEventCount 
  
      Duration batchInterval = new Duration(2000);
  
 -    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
 +    JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
-             System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+             System.getenv("SPARK_HOME"),
+             JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class));
 -
 -    JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
 +    FlumeFunctions flumeFunc = new FlumeFunctions(ssc);
 +    JavaDStream<SparkFlumeEvent> flumeStream = flumeFunc.flumeStream("localhost", port);
  
      flumeStream.count();
  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
----------------------------------------------------------------------
diff --cc external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
index ba33320,0000000..74840f6
mode 100644,000000..100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@@ -1,86 -1,0 +1,86 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.spark.streaming.flume
 +
 +import scala.collection.JavaConversions._
 +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
 +
 +import java.net.InetSocketAddress
 +import java.nio.ByteBuffer
 +import java.nio.charset.Charset
 +
 +import org.apache.avro.ipc.NettyTransceiver
 +import org.apache.avro.ipc.specific.SpecificRequestor
 +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol}
 +
 +import org.apache.spark.storage.StorageLevel
 +import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase}
 +import org.apache.spark.streaming.util.ManualClock
 +
 +class FlumeStreamSuite extends TestSuiteBase {
 +
 +  val testPort = 9999
 +
 +  test("flume input stream") {
 +    // Set up the streaming context and input streams
-     val ssc = new StreamingContext(master, framework, batchDuration)
++    val ssc = new StreamingContext(conf, batchDuration)
 +    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
 +    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
 +      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
 +    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
 +    ssc.registerOutputStream(outputStream)
 +    ssc.start()
 +
 +    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
 +    val input = Seq(1, 2, 3, 4, 5)
 +    Thread.sleep(1000)
 +    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
 +    val client = SpecificRequestor.getClient(
 +      classOf[AvroSourceProtocol], transceiver)
 +
 +    for (i <- 0 until input.size) {
 +      val event = new AvroFlumeEvent
 +      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
 +      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
 +      client.append(event)
 +      Thread.sleep(500)
 +      clock.addToTime(batchDuration.milliseconds)
 +    }
 +
 +    val startTime = System.currentTimeMillis()
 +    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
 +      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
 +      Thread.sleep(100)
 +    }
 +    Thread.sleep(1000)
 +    val timeTaken = System.currentTimeMillis() - startTime
 +    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
 +    logInfo("Stopping context")
 +    ssc.stop()
 +
 +    val decoder = Charset.forName("UTF-8").newDecoder()
 +
 +    assert(outputBuffer.size === input.length)
 +    for (i <- 0 until outputBuffer.size) {
 +      assert(outputBuffer(i).size === 1)
 +      val str = decoder.decode(outputBuffer(i).head.event.getBody)
 +      assert(str.toString === input(i).toString)
 +      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/project/SparkBuild.scala
----------------------------------------------------------------------
diff --cc project/SparkBuild.scala
index b7aa732,a6c560d..84becf6
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@@ -82,34 -89,12 +86,33 @@@ object SparkBuild extends Build 
    }
  
    // Conditionally include the yarn sub-project
-   lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core)
+   lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core)
+   lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core)
  
-   //lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
- 
-   lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
-   lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+   lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]()
+   lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]()
  
 +  lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) 
 +    .dependsOn(streaming % "compile->compile;test->test")
 +
 +  lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings)
 +    .dependsOn(streaming % "compile->compile;test->test")
 +
 +  lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings)
 +    .dependsOn(streaming % "compile->compile;test->test")
 +  
 +  lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings)
 +    .dependsOn(streaming % "compile->compile;test->test")
 +  
 +  lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings)
 +    .dependsOn(streaming % "compile->compile;test->test")
 +
 +  lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
 +  lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
 +  
 +  lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
 +    .dependsOn(core, mllib, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
 +
    // Everything except assembly, tools and examples belong to packageProjects
    lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
  
@@@ -192,10 -178,11 +196,10 @@@
          Some("sonatype-staging"  at nexus + "service/local/staging/deploy/maven2")
      },
  
 -*/
 -
 +    */
  
      libraryDependencies ++= Seq(
-         "io.netty"          % "netty-all"       % "4.0.0.CR1",
+         "io.netty"          % "netty-all"       % "4.0.13.Final",
          "org.eclipse.jetty" % "jetty-server"    % "7.6.8.v20121106",
          /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */
          "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"),

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 96f57cb,b3a7cf0..693cb7f
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@@ -23,25 -40,16 +23,23 @@@ import scala.reflect.ClassTa
  
  import java.io.InputStream
  import java.util.concurrent.atomic.AtomicInteger
- import java.util.UUID
  
 +import akka.actor.Props
 +import akka.actor.SupervisorStrategy
  import org.apache.hadoop.io.LongWritable
  import org.apache.hadoop.io.Text
  import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
  import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
  import org.apache.hadoop.fs.Path
  
 -import twitter4j.Status
 -import twitter4j.auth.Authorization
 +import org.apache.spark._
 +import org.apache.spark.rdd.RDD
 +import org.apache.spark.storage.StorageLevel
 +import org.apache.spark.util.MetadataCleaner
 +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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
diff --cc streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index ea4a0fe,7dec4b3..7068f32
--- 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,21 -17,27 +17,23 @@@
  
  package org.apache.spark.streaming.api.java
  
 -import java.io.InputStream
 -import java.lang.{Integer => JInt}
 -import java.util.{List => JList, Map => JMap}
+ 
  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 akka.util.ByteString
 -import akka.zeromq.Subscribe
  import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
- import akka.actor.Props
- import akka.actor.SupervisorStrategy
 -import twitter4j.Status
 -import twitter4j.auth.Authorization
  
+ 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.dstream._
  import org.apache.spark.streaming.scheduler.StreamingListener
  
  /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --cc streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index 6218795,d53d433..0d2145d
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@@ -17,17 -17,24 +17,19 @@@
  
  package org.apache.spark.streaming;
  
 -import com.google.common.base.Optional;
 -import com.google.common.collect.Lists;
 -import com.google.common.collect.Maps;
 -import com.google.common.io.Files;
 -
 -import kafka.serializer.StringDecoder;
 +import scala.Tuple2;
  
 -import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 -import org.apache.spark.SparkConf;
 -import org.apache.spark.streaming.api.java.JavaDStreamLike;
+ import org.junit.After;
  import org.junit.Assert;
 -import org.junit.Before;
  import org.junit.Test;
 +import java.io.*;
 +import java.util.*;
  
 -import scala.Tuple2;
 -import twitter4j.Status;
 +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;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
diff --cc streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 0cffed6,5185954..a8e0532
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@@ -49,9 -56,9 +49,9 @@@ class InputStreamsSuite extends TestSui
      testServer.start()
  
      // Set up the streaming context and input streams
-     val ssc = new StreamingContext(master, framework, batchDuration)
 -    val ssc = new StreamingContext(new SparkContext(conf), 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)
@@@ -92,13 -99,62 +92,13 @@@
    }
  
  
 -  test("flume input stream") {
 -    // Set up the streaming context and input streams
 -    val ssc = new StreamingContext(new SparkContext(conf), batchDuration)
 -    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
 -    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
 -      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
 -    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
 -    ssc.registerOutputStream(outputStream)
 -    ssc.start()
 -
 -    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
 -    val input = Seq(1, 2, 3, 4, 5)
 -    Thread.sleep(1000)
 -    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
 -    val client = SpecificRequestor.getClient(
 -      classOf[AvroSourceProtocol], transceiver)
 -
 -    for (i <- 0 until input.size) {
 -      val event = new AvroFlumeEvent
 -      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
 -      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
 -      client.append(event)
 -      Thread.sleep(500)
 -      clock.addToTime(batchDuration.milliseconds)
 -    }
 -
 -    val startTime = System.currentTimeMillis()
 -    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
 -      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
 -      Thread.sleep(100)
 -    }
 -    Thread.sleep(1000)
 -    val timeTaken = System.currentTimeMillis() - startTime
 -    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
 -    logInfo("Stopping context")
 -    ssc.stop()
 -
 -    val decoder = Charset.forName("UTF-8").newDecoder()
 -
 -    assert(outputBuffer.size === input.length)
 -    for (i <- 0 until outputBuffer.size) {
 -      assert(outputBuffer(i).size === 1)
 -      val str = decoder.decode(outputBuffer(i).head.event.getBody)
 -      assert(str.toString === input(i).toString)
 -      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
 -    }
 -  }
 -
 -
    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(new SparkContext(conf), 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)
@@@ -150,7 -206,7 +150,7 @@@
      testServer.start()
  
      // Set up the streaming context and input streams
-     val ssc = new StreamingContext(master, framework, batchDuration)
 -    val ssc = new StreamingContext(new SparkContext(conf), 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 -249,21 +193,7 @@@
      }
    }
  
 -  test("kafka input stream") {
 -    val ssc = new StreamingContext(new SparkContext(conf), batchDuration)
 -    val topics = Map("my-topic" -> 1)
 -    val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
 -    val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK)
 -
 -    // Test specifying decoder
 -    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
 -    val test3 = ssc.kafkaStream[
 -      String,
 -      String,
 -      kafka.serializer.StringDecoder,
 -      kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
 -  }
+ 
    test("multi-thread receiver") {
      // set up the test receiver
      val numThreads = 10
@@@ -202,7 -273,7 +203,7 @@@
      MultiThreadTestReceiver.haveAllThreadsFinished = false
  
      // set up the network stream using the test receiver
-     val ssc = new StreamingContext(master, framework, batchDuration)
 -    val ssc = new StreamingContext(new SparkContext(conf), 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]]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/3b4c4c7f/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
diff --cc streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index f56c046,33464bc..b20d02f
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@@ -136,16 -142,17 +142,13 @@@ trait TestSuiteBase extends FunSuite wi
    // 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) {
 -    //  System.setProperty(
 -    //    "spark.streaming.clock",
 -    //    "org.apache.spark.streaming.util.ManualClock"
 -    //  )
 -    //} else {
 -    //  System.clearProperty("spark.streaming.clock")
 -    //}
      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 -175,9 +171,8 @@@
        operation: DStream[U] => DStream[V],
        numPartitions: Int = numInputPartitions
      ): StreamingContext = {
- 
 -    val sc = new SparkContext(conf)
      // Create StreamingContext
-     val ssc = new StreamingContext(master, framework, batchDuration)
 -    val ssc = new StreamingContext(sc, batchDuration)
++    val ssc = new StreamingContext(conf, batchDuration)
      if (checkpointDir != null) {
        ssc.checkpoint(checkpointDir)
      }
@@@ -194,9 -201,9 +196,8 @@@
        input2: Seq[Seq[V]],
        operation: (DStream[U], DStream[V]) => DStream[W]
      ): StreamingContext = {
- 
 -    val sc = new SparkContext(conf)
      // Create StreamingContext
-     val ssc = new StreamingContext(master, framework, batchDuration)
 -    val ssc = new StreamingContext(sc, batchDuration)
++    val ssc = new StreamingContext(conf, batchDuration)
      if (checkpointDir != null) {
        ssc.checkpoint(checkpointDir)
      }


[12/13] git commit: Fixed examples/pom.xml and run-example based on Patrick's suggestions.

Posted by pw...@apache.org.
Fixed examples/pom.xml and run-example based on Patrick's suggestions.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/8f02f1c3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/8f02f1c3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/8f02f1c3

Branch: refs/heads/master
Commit: 8f02f1c3d45ee553ed6bec2dc81fbae4435274fc
Parents: aa99f22
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Jan 7 11:02:29 2014 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Jan 7 11:02:29 2014 -0800

----------------------------------------------------------------------
 bin/run-example  | 7 +------
 examples/pom.xml | 7 +------
 2 files changed, 2 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8f02f1c3/bin/run-example
----------------------------------------------------------------------
diff --git a/bin/run-example b/bin/run-example
index dfb4bf7..a94913d 100755
--- a/bin/run-example
+++ b/bin/run-example
@@ -45,20 +45,15 @@ fi
 EXAMPLES_DIR="$FWDIR"/examples
 SPARK_EXAMPLES_JAR=""
 if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
-  # Use the JAR from the SBT build
   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
 fi
-if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
-  # Use the JAR from the Maven build
-  # TODO: this also needs to become an assembly!
-  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
-fi
 if [[ -z $SPARK_EXAMPLES_JAR ]]; then
   echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
   echo "You need to build Spark with sbt assembly before running this program" >&2
   exit 1
 fi
 
+
 # Since the examples JAR ideally shouldn't include spark-core (that dependency should be
 # "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
 CLASSPATH=`$FWDIR/bin/compute-classpath.sh`

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/8f02f1c3/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 1839667..9c77640 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -75,31 +75,26 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming-twitter_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming-kafka_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming-zeromq_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -189,7 +184,7 @@
         <artifactId>maven-shade-plugin</artifactId>
         <configuration>
           <shadedArtifactAttached>false</shadedArtifactAttached>
-          <outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
+          <outputFile>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
           <artifactSet>
             <includes>
               <include>*:*</include>


[10/13] Removed XYZFunctions and added XYZUtils as a common Scala and Java interface for creating XYZ streams.

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
deleted file mode 100644
index f4c75ab..0000000
--- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.zeromq
-
-import scala.reflect.ClassTag
-
-import akka.actor.{Props, SupervisorStrategy}
-import akka.util.ByteString
-import akka.zeromq.Subscribe
-
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming._
-import org.apache.spark.streaming.receivers._
-
-/**
- * Extra ZeroMQ input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
- * through implicit conversions. Import org.apache.spark.streaming.zeromq._ to use these functions.
- */
-class ZeroMQFunctions(ssc: StreamingContext) {
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
-   *                       and each frame has sequence of byte thus it needs the converter
-   *                       (which might be deserializer of bytes) to translate from sequence
-   *                       of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-  def zeroMQStream[T: ClassTag](
-      publisherUrl: String,
-      subscribe: Subscribe,
-      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2,
-      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
-    ): DStream[T] = {
-    ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
-        "ZeroMQReceiver", storageLevel, supervisorStrategy)
-  }
-}
-  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
new file mode 100644
index 0000000..546d9df
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+import akka.actor.{Props, SupervisorStrategy}
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object ZeroMQUtils {
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param ssc            StreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
+   *                       and each frame has sequence of byte thus it needs the converter
+   *                       (which might be deserializer of bytes) to translate from sequence
+   *                       of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel   RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+   */
+  def createStream[T: ClassTag](
+      ssc: StreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2,
+      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
+    ): DStream[T] = {
+    ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
+      "ZeroMQReceiver", storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote ZeroMQ publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel   RDD storage level.
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param jssc           JavaStreamingContext object
+   * @param publisherUrl   Url of remote zeromq publisher
+   * @param subscribe      Topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   */
+  def createStream[T](
+      jssc: JavaStreamingContext,
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    createStream[T](jssc.ssc, publisherUrl, subscribe, fn)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
deleted file mode 100644
index dc27178..0000000
--- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-package object zeromq {
-  implicit def sscToZeroMQFunctions(ssc: StreamingContext) = new ZeroMQFunctions(ssc)
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
index b020ae4..d2361e1 100644
--- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
@@ -17,13 +17,10 @@
 
 package org.apache.spark.streaming.zeromq;
 
-import org.apache.spark.streaming.api.java.zeromq.ZeroMQFunctions;
 import org.junit.Test;
-
 import akka.actor.SupervisorStrategy;
 import akka.util.ByteString;
 import akka.zeromq.Subscribe;
-
 import org.apache.spark.api.java.function.Function;
 import org.apache.spark.storage.StorageLevel;
 import org.apache.spark.streaming.LocalJavaStreamingContext;
@@ -33,7 +30,6 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
 
   @Test // tests the API, does not actually test data receiving
   public void testZeroMQStream() {
-    ZeroMQFunctions zeromqFunc = new ZeroMQFunctions(ssc);
     String publishUrl = "abc";
     Subscribe subscribe = new Subscribe((ByteString)null);
     Function<byte[][], Iterable<String>> bytesToObjects = new Function<byte[][], Iterable<String>>() {
@@ -43,11 +39,12 @@ public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
       }
     };
 
-    JavaDStream<String> test1 = zeromqFunc.<String>zeroMQStream(
-      publishUrl, subscribe, bytesToObjects);
-    JavaDStream<String> test2 = zeromqFunc.<String>zeroMQStream(
-      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
-    JavaDStream<String> test3 = zeromqFunc.<String>zeroMQStream(
-      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy());
+    JavaDStream<String> test1 = ZeroMQUtils.<String>createStream(
+      ssc, publishUrl, subscribe, bytesToObjects);
+    JavaDStream<String> test2 = ZeroMQUtils.<String>createStream(
+      ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<String> test3 = ZeroMQUtils.<String>createStream(
+      ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(),
+      SupervisorStrategy.defaultStrategy());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/aa99f226/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
index 5adcdb8..4193b8a 100644
--- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -33,10 +33,10 @@ class ZeroMQStreamSuite extends TestSuiteBase {
     val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]]
 
     // tests the API, does not actually test data receiving
-    val test1 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects)
-    val test2 = ssc.zeroMQStream(
-      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2)
-    val test3 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects,
+    val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects)
+    val test2 = ZeroMQUtils.createStream(
+      ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects,
       StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy)
 
     // TODO: Actually test data receiving


[02/13] Refactored kafka, flume, zeromq, mqtt as separate external projects, with their own self-contained scala API, java API, scala unit tests and java unit tests. Updated examples to use the external projects.

Posted by pw...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
new file mode 100644
index 0000000..d7f6d35
--- /dev/null
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+import twitter4j.auth.{NullAuthorization, Authorization}
+
+class TwitterStreamSuite extends TestSuiteBase {
+
+  test("kafka input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val filters = Seq("filter1", "filter2")
+    val authorization: Authorization = NullAuthorization.getInstance()
+
+    // tests the API, does not actually test data receiving
+    val test1 = ssc.twitterStream(None)
+    val test2 = ssc.twitterStream(None, filters)
+    val test3 = ssc.twitterStream(None, filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test4 = ssc.twitterStream(Some(authorization))
+    val test5 = ssc.twitterStream(Some(authorization), filters)
+    val test6 = ssc.twitterStream(Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+    // Note that actually testing the data receiving is hard as authentication keys are
+    // necessary for accessing Twitter live stream
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
new file mode 100644
index 0000000..dc5d1f0
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/JavaStreamingContextWithZeroMQ.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import akka.actor.SupervisorStrategy
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+/**
+ * Subclass of [[org.apache.spark.streaming.api.java.JavaStreamingContext]] that has extra
+ * functions for creating ZeroMQ input streams.
+ */
+class JavaStreamingContextWithZeroMQ(javaStreamingContext: JavaStreamingContext)
+  extends JavaStreamingContext(javaStreamingContext.ssc) {
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote ZeroMQ publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel RDD storage level.
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   */
+  def zeroMQStream[T](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+    ): JavaDStream[T] = {
+    implicit val cm: ClassTag[T] =
+      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
new file mode 100644
index 0000000..f4c75ab
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQFunctions.scala
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+
+import akka.actor.{Props, SupervisorStrategy}
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.receivers._
+
+/**
+ * Extra ZeroMQ input stream functions available on [[org.apache.spark.streaming.StreamingContext]]
+ * through implicit conversions. Import org.apache.spark.streaming.zeromq._ to use these functions.
+ */
+class ZeroMQFunctions(ssc: StreamingContext) {
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
+   *                       and each frame has sequence of byte thus it needs the converter
+   *                       (which might be deserializer of bytes) to translate from sequence
+   *                       of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel RDD storage level. Defaults to memory-only.
+   */
+  def zeroMQStream[T: ClassTag](
+      publisherUrl: String,
+      subscribe: Subscribe,
+      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2,
+      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
+    ): DStream[T] = {
+    ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
+        "ZeroMQReceiver", storageLevel, supervisorStrategy)
+  }
+}
+  

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
new file mode 100644
index 0000000..769761e
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+
+import akka.actor.Actor
+import akka.util.ByteString
+import akka.zeromq._
+
+import org.apache.spark.Logging
+import org.apache.spark.streaming.receivers._
+
+/**
+ * A receiver to subscribe to ZeroMQ stream.
+ */
+private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String,
+  subscribe: Subscribe,
+  bytesToObjects: Seq[ByteString] ⇒ Iterator[T])
+  extends Actor with Receiver with Logging {
+
+  override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self),
+    Connect(publisherUrl), subscribe)
+
+  def receive: Receive = {
+
+    case Connecting ⇒ logInfo("connecting ...")
+
+    case m: ZMQMessage ⇒
+      logDebug("Received message for:" + m.frame(0))
+
+      //We ignore first frame for processing as it is the topic
+      val bytes = m.frames.tail
+      pushBlock(bytesToObjects(bytes))
+
+    case Closed ⇒ logInfo("received closed ")
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
new file mode 100644
index 0000000..dc27178
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+package object zeromq {
+  implicit def sscToZeroMQFunctions(ssc: StreamingContext) = new ZeroMQFunctions(ssc)
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
new file mode 100644
index 0000000..96af7d7
--- /dev/null
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq;
+
+import org.junit.Test;
+
+import akka.actor.SupervisorStrategy;
+import akka.util.ByteString;
+import akka.zeromq.Subscribe;
+
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
+
+  @Test // tests the API, does not actually test data receiving
+  public void testZeroMQStream() {
+    JavaStreamingContextWithZeroMQ sscWithZeroMQ = new JavaStreamingContextWithZeroMQ(ssc);
+    String publishUrl = "abc";
+    Subscribe subscribe = new Subscribe((ByteString)null);
+    Function<byte[][], Iterable<String>> bytesToObjects = new Function<byte[][], Iterable<String>>() {
+      @Override
+      public Iterable<String> call(byte[][] bytes) throws Exception {
+        return null;
+      }
+    };
+
+    JavaDStream<String> test1 = sscWithZeroMQ.<String>zeroMQStream(
+      publishUrl, subscribe, bytesToObjects);
+    JavaDStream<String> test2 = sscWithZeroMQ.<String>zeroMQStream(
+      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
+    JavaDStream<String> test3 = sscWithZeroMQ.<String>zeroMQStream(
+      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy());
+
+    // To verify that JavaStreamingContextWithKafka is also StreamingContext
+    JavaDStream<String> socketStream = sscWithZeroMQ.socketTextStream("localhost", 9999);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties
new file mode 100644
index 0000000..063529a
--- /dev/null
+++ b/external/zeromq/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
----------------------------------------------------------------------
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
new file mode 100644
index 0000000..5adcdb8
--- /dev/null
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import akka.actor.SupervisorStrategy
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+
+class ZeroMQStreamSuite extends TestSuiteBase {
+
+  test("zeromq input stream") {
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val publishUrl = "abc"
+    val subscribe = new Subscribe(null.asInstanceOf[ByteString])
+    val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]]
+
+    // tests the API, does not actually test data receiving
+    val test1 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects)
+    val test2 = ssc.zeroMQStream(
+      publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2)
+    val test3 = ssc.zeroMQStream(publishUrl, subscribe, bytesToObjects,
+      StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy)
+
+    // TODO: Actually test data receiving
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index f9ff781..4e92d2a 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -49,9 +49,6 @@ object SparkBuild extends Build {
   lazy val repl = Project("repl", file("repl"), settings = replSettings)
     .dependsOn(core, bagel, mllib)
 
-  lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
-    .dependsOn(core, mllib, bagel, streaming, externalTwitter)
-
   lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
 
   lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core)
@@ -60,8 +57,6 @@ object SparkBuild extends Build {
 
   lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
 
-  lazy val externalTwitter = Project("streaming-twitter", file("external/twitter"), settings = twitterSettings) dependsOn(streaming)
-
   lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
     .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
 
@@ -94,10 +89,31 @@ object SparkBuild extends Build {
   lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
   lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
 
+  lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) 
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+  
+  lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+  
+  lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings)
+    .dependsOn(streaming % "compile->compile;test->test")
+
+  lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+  lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+  
+  lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
+    .dependsOn(core, mllib, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
+
   // Everything except assembly, tools and examples belong to packageProjects
   lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib) ++ maybeYarnRef
 
-  lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj)
+  lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) 
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization       := "org.apache.spark",
@@ -167,7 +183,7 @@ object SparkBuild extends Build {
       </issueManagement>
     ),
 
-/*
+    /*
     publishTo <<= version { (v: String) =>
       val nexus = "https://oss.sonatype.org/"
       if (v.trim.endsWith("SNAPSHOT"))
@@ -176,8 +192,7 @@ object SparkBuild extends Build {
         Some("sonatype-staging"  at nexus + "service/local/staging/deploy/maven2")
     },
 
-*/
-
+    */
 
     libraryDependencies ++= Seq(
         "io.netty"          % "netty-all"       % "4.0.0.CR1",
@@ -264,7 +279,6 @@ object SparkBuild extends Build {
    libraryDependencies <+= scalaVersion(v => "org.scala-lang"  % "scala-reflect"  % v )
   )
 
-
   def examplesSettings = sharedSettings ++ Seq(
     name := "spark-examples",
     libraryDependencies ++= Seq(
@@ -302,21 +316,10 @@ object SparkBuild extends Build {
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
     resolvers ++= Seq(
-      "Eclipse Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/",
       "Apache repo" at "https://repository.apache.org/content/repositories/releases"
     ),
-
     libraryDependencies ++= Seq(
-      "org.apache.flume"        % "flume-ng-sdk"     % "1.2.0" % "compile"     excludeAll(excludeNetty, excludeSnappy),
-      "com.sksamuel.kafka"     %% "kafka"            % "0.8.0-beta1"
-        exclude("com.sun.jdmk", "jmxtools")
-        exclude("com.sun.jmx", "jmxri")
-        exclude("net.sf.jopt-simple", "jopt-simple")
-        excludeAll(excludeNetty),
-      "org.eclipse.paho"        % "mqtt-client"      % "0.4.0",
-      "com.github.sgroschupf"   % "zkclient"         % "0.1"                   excludeAll(excludeNetty),
-      // "org.twitter4j"           % "twitter4j-stream" % "3.0.3"                 excludeAll(excludeNetty),
-      "org.spark-project.akka" %% "akka-zeromq"      % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
+      "commons-io" % "commons-io" % "2.4" 
     )
   )
 
@@ -331,8 +334,8 @@ object SparkBuild extends Build {
   def yarnEnabledSettings = Seq(
     libraryDependencies ++= Seq(
       // Exclude rule required for all ?
-      "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
-      "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+      "org.apache.hadoop" % "hadoop-client"      % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+      "org.apache.hadoop" % "hadoop-yarn-api"    % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
       "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
       "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib)
     )
@@ -358,9 +361,45 @@ object SparkBuild extends Build {
   )
 
   def twitterSettings() = streamingSettings ++ Seq(
-    name := "spark-twitter",
+    name := "spark-streaming-twitter",
     libraryDependencies ++= Seq(
       "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty)
     )
   )
+  
+  def kafkaSettings() = streamingSettings ++ Seq(
+    name := "spark-streaming-kafka",
+    libraryDependencies ++= Seq(
+      "com.github.sgroschupf"    % "zkclient"   % "0.1"          excludeAll(excludeNetty),
+      "com.sksamuel.kafka"      %% "kafka"      % "0.8.0-beta1"
+        exclude("com.sun.jdmk", "jmxtools")
+        exclude("com.sun.jmx", "jmxri")
+        exclude("net.sf.jopt-simple", "jopt-simple")
+        excludeAll(excludeNetty)
+    )
+  )
+  
+  def flumeSettings() = streamingSettings ++ Seq(
+    name := "spark-streaming-flume",
+    libraryDependencies ++= Seq(
+      "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy)
+    )
+  )
+
+  def zeromqSettings() = streamingSettings ++ Seq(
+    name := "spark-streaming-zeromq",
+    libraryDependencies ++= Seq(
+      "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
+    )
+  )
+
+  def mqttSettings() = streamingSettings ++ Seq(
+    name := "spark-streaming-mqtt",
+    resolvers ++= Seq(
+      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
+    ),
+    libraryDependencies ++= Seq(
+      "org.eclipse.paho" % "mqtt-client" % "0.4.0"
+    )
+  )
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
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 25b9b70..41898b9 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -17,21 +17,6 @@
 
 package org.apache.spark.streaming
 
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-
-import org.apache.spark.streaming.dstream._
-
-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
 import scala.reflect.ClassTag
@@ -40,15 +25,22 @@ import java.io.InputStream
 import java.util.concurrent.atomic.AtomicInteger
 import java.util.UUID
 
+import org.apache.spark._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.streaming.receivers._
+import org.apache.spark.streaming.scheduler._
+
 import org.apache.hadoop.io.LongWritable
 import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
 import org.apache.hadoop.fs.Path
-//import twitter4j.Status
-//import twitter4j.auth.Authorization
-import org.apache.spark.streaming.scheduler._
-import akka.util.ByteString
+
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -224,74 +216,6 @@ class StreamingContext private (
   }
 
   /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
-   *                       and each frame has sequence of byte thus it needs the converter
-   *                       (which might be deserializer of bytes) to translate from sequence
-   *                       of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-  def zeroMQStream[T: ClassTag](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
-      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
-    ): DStream[T] = {
-    actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
-        "ZeroMQReceiver", storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
-   */
-  def kafkaStream(
-      zkQuorum: String,
-      groupId: String,
-      topics: Map[String, Int],
-      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
-    ): DStream[(String, String)] = {
-    val kafkaParams = Map[String, String](
-      "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
-      "zookeeper.connection.timeout.ms" -> "10000")
-    kafkaStream[String, String, kafka.serializer.StringDecoder, kafka.serializer.StringDecoder](
-      kafkaParams,
-      topics,
-      storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages from a Kafka Broker.
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def kafkaStream[
-    K: ClassTag,
-    V: ClassTag,
-    U <: kafka.serializer.Decoder[_]: Manifest,
-    T <: kafka.serializer.Decoder[_]: Manifest](
-      kafkaParams: Map[String, String],
-      topics: Map[String, Int],
-      storageLevel: StorageLevel
-    ): DStream[(K, V)] = {
-    val inputStream = new KafkaInputDStream[K, V, U, T](this, kafkaParams, topics, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-
-  /**
    * Create a input stream from TCP source hostname:port. Data is received using
    * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited
    * lines.
@@ -330,22 +254,6 @@ class StreamingContext private (
   }
 
   /**
-   * Create a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream (
-      hostname: String,
-      port: Int,
-      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
-    ): DStream[SparkFlumeEvent] = {
-    val inputStream = new FlumeInputDStream[SparkFlumeEvent](this, hostname, port, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
-
-  /**
    * Create a input stream from network source hostname:port, where data is received
    * as serialized blocks (serialized using the Spark's serializer) that can be directly
    * pushed into the block manager without deserializing them. This is the most efficient
@@ -467,21 +375,6 @@ class StreamingContext private (
     inputStream
   }
 
-/**
-   * Create an input stream that receives messages pushed by a mqtt publisher.
-   * @param brokerUrl Url of remote mqtt publisher
-   * @param topic topic name to subscribe to
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-
-  def mqttStream(
-    brokerUrl: String,
-    topic: String,
-    storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[String] = {
-    val inputStream = new MQTTInputDStream[String](this, brokerUrl, topic, storageLevel)
-    registerInputStream(inputStream)
-    inputStream
-  }
   /**
    * Create a unified DStream from multiple DStreams of the same type and same slide duration.
    */

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
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 b32cfbb..ea4a0fe 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,28 +17,21 @@
 
 package org.apache.spark.streaming.api.java
 
-import java.lang.{Integer => JInt}
-import java.io.InputStream
-import java.util.{Map => JMap, List => JList}
-
 import scala.collection.JavaConversions._
 import scala.reflect.ClassTag
 
+import java.io.InputStream
+import java.util.{Map => JMap, List => JList}
+
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-//import twitter4j.Status
 import akka.actor.Props
 import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-import akka.util.ByteString
-
-//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.{JavaPairRDD, JavaSparkContext, JavaRDD}
 import org.apache.spark.streaming._
-import org.apache.spark.streaming.dstream._
 import org.apache.spark.streaming.scheduler.StreamingListener
 
 /**
@@ -134,81 +127,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
 
   /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   */
-  def kafkaStream(
-    zkQuorum: String,
-    groupId: String,
-    topics: JMap[String, JInt])
-  : JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
-      StorageLevel.MEMORY_ONLY_SER_2)
-
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
-   * @param groupId The group id for this consumer.
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   *               in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   *
-   */
-  def kafkaStream(
-    zkQuorum: String,
-    groupId: String,
-    topics: JMap[String, JInt],
-    storageLevel: StorageLevel)
-  : JavaPairDStream[String, String] = {
-    implicit val cmt: ClassTag[String] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
-    ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
-      storageLevel)
-  }
-
-  /**
-   * Create an input stream that pulls messages form a Kafka Broker.
-   * @param keyTypeClass Key type of RDD
-   * @param valueTypeClass value type of RDD
-   * @param keyDecoderClass Type of kafka key decoder
-   * @param valueDecoderClass Type of kafka value decoder
-   * @param kafkaParams Map of kafka configuration paramaters.
-   *                    See: http://kafka.apache.org/configuration.html
-   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
-   * in its own thread.
-   * @param storageLevel RDD storage level. Defaults to memory-only
-   */
-  def kafkaStream[K, V, U <: kafka.serializer.Decoder[_], T <: kafka.serializer.Decoder[_]](
-    keyTypeClass: Class[K],
-    valueTypeClass: Class[V],
-    keyDecoderClass: Class[U],
-    valueDecoderClass: Class[T],
-    kafkaParams: JMap[String, String],
-    topics: JMap[String, JInt],
-    storageLevel: StorageLevel)
-  : JavaPairDStream[K, V] = {
-    implicit val keyCmt: ClassTag[K] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
-    implicit val valueCmt: ClassTag[V] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
-
-    implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
-    implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
-
-    ssc.kafkaStream[K, V, U, T](
-      kafkaParams.toMap,
-      Map(topics.mapValues(_.intValue()).toSeq: _*),
-      storageLevel)
-  }
-
-  /**
    * Create a input stream from network source hostname:port. Data is received using
    * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
    * lines.
@@ -319,98 +237,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   }
 
   /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
-    JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port, storageLevel)
-  }
-
-
-  /**
-   * Creates a input stream from a Flume source.
-   * @param hostname Hostname of the slave machine to which the flume data will be sent
-   * @param port     Port of the slave machine to which the flume data will be sent
-   */
-  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
-    ssc.flumeStream(hostname, port)
-  }
-  /*
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization object
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   * @param filters Set of filter strings to get only those tweets that match them
-   * @param storageLevel Storage level to use for storing the received objects
-   */
-  def twitterStream(
-      filters: Array[String],
-      storageLevel: StorageLevel
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters, storageLevel)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      twitterAuth: Authorization,
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth), filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   * @param filters Set of filter strings to get only those tweets that match them
-   */
-  def twitterStream(
-      filters: Array[String]
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(None, filters)
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter.
-   * @param twitterAuth Twitter4J Authorization
-   */
-  def twitterStream(
-      twitterAuth: Authorization
-    ): JavaDStream[Status] = {
-    ssc.twitterStream(Some(twitterAuth))
-  }
-
-  /**
-   * Create a input stream that returns tweets received from Twitter using Twitter4J's default
-   * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
-   * .consumerSecret, .accessToken and .accessTokenSecret to be set.
-   */
-  def twitterStream(): JavaDStream[Status] = {
-    ssc.twitterStream()
-  }
-  */
-  /**
    * Create an input stream with any arbitrary user implemented actor receiver.
    * @param props Props object defining creation of the actor
    * @param name Name of the actor
@@ -473,70 +299,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   }
 
   /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel  Storage level to use for storing the received objects
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
-      storageLevel: StorageLevel,
-      supervisorStrategy: SupervisorStrategy
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   * @param storageLevel RDD storage level. Defaults to memory-only.
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
-      storageLevel: StorageLevel
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
-  }
-
-  /**
-   * Create an input stream that receives messages pushed by a zeromq publisher.
-   * @param publisherUrl Url of remote zeromq publisher
-   * @param subscribe topic to subscribe to
-   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
-   *                       of byte thus it needs the converter(which might be deserializer of bytes)
-   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
-   *                       and sub sequence refer to its payload.
-   */
-  def zeroMQStream[T](
-      publisherUrl:String,
-      subscribe: Subscribe,
-      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
-    ): JavaDStream[T] = {
-    implicit val cm: ClassTag[T] =
-      implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
-    def fn(x: Seq[ByteString]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
-    ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
-  }
-
-  /**
    * Registers an output stream that will be computed every interval
    */
   def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
deleted file mode 100644
index 60d7917..0000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.dstream
-
-import java.net.InetSocketAddress
-import java.io.{ObjectInput, ObjectOutput, Externalizable}
-import java.nio.ByteBuffer
-
-import scala.collection.JavaConversions._
-import scala.reflect.ClassTag
-
-import org.apache.flume.source.avro.AvroSourceProtocol
-import org.apache.flume.source.avro.AvroFlumeEvent
-import org.apache.flume.source.avro.Status
-import org.apache.avro.ipc.specific.SpecificResponder
-import org.apache.avro.ipc.NettyServer
-
-import org.apache.spark.streaming.StreamingContext
-import org.apache.spark.util.Utils
-import org.apache.spark.storage.StorageLevel
-
-private[streaming]
-class FlumeInputDStream[T: ClassTag](
-  @transient ssc_ : StreamingContext,
-  host: String,
-  port: Int,
-  storageLevel: StorageLevel
-) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
-
-  override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = {
-    new FlumeReceiver(host, port, storageLevel)
-  }
-}
-
-/**
- * A wrapper class for AvroFlumeEvent's with a custom serialization format.
- *
- * This is necessary because AvroFlumeEvent uses inner data structures
- * which are not serializable.
- */
-class SparkFlumeEvent() extends Externalizable {
-  var event : AvroFlumeEvent = new AvroFlumeEvent()
-
-  /* De-serialize from bytes. */
-  def readExternal(in: ObjectInput) {
-    val bodyLength = in.readInt()
-    val bodyBuff = new Array[Byte](bodyLength)
-    in.read(bodyBuff)
-
-    val numHeaders = in.readInt()
-    val headers = new java.util.HashMap[CharSequence, CharSequence]
-
-    for (i <- 0 until numHeaders) {
-      val keyLength = in.readInt()
-      val keyBuff = new Array[Byte](keyLength)
-      in.read(keyBuff)
-      val key : String = Utils.deserialize(keyBuff)
-
-      val valLength = in.readInt()
-      val valBuff = new Array[Byte](valLength)
-      in.read(valBuff)
-      val value : String = Utils.deserialize(valBuff)
-
-      headers.put(key, value)
-    }
-
-    event.setBody(ByteBuffer.wrap(bodyBuff))
-    event.setHeaders(headers)
-  }
-
-  /* Serialize to bytes. */
-  def writeExternal(out: ObjectOutput) {
-    val body = event.getBody.array()
-    out.writeInt(body.length)
-    out.write(body)
-
-    val numHeaders = event.getHeaders.size()
-    out.writeInt(numHeaders)
-    for ((k, v) <- event.getHeaders) {
-      val keyBuff = Utils.serialize(k.toString)
-      out.writeInt(keyBuff.length)
-      out.write(keyBuff)
-      val valBuff = Utils.serialize(v.toString)
-      out.writeInt(valBuff.length)
-      out.write(valBuff)
-    }
-  }
-}
-
-private[streaming] object SparkFlumeEvent {
-  def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = {
-    val event = new SparkFlumeEvent
-    event.event = in
-    event
-  }
-}
-
-/** A simple server that implements Flume's Avro protocol. */
-private[streaming]
-class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol {
-  override def append(event : AvroFlumeEvent) : Status = {
-    receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)
-    Status.OK
-  }
-
-  override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = {
-    events.foreach (event =>
-      receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event))
-    Status.OK
-  }
-}
-
-/** A NetworkReceiver which listens for events using the
-  * Flume Avro interface.*/
-private[streaming]
-class FlumeReceiver(
-    host: String,
-    port: Int,
-    storageLevel: StorageLevel
-  ) extends NetworkReceiver[SparkFlumeEvent] {
-
-  lazy val blockGenerator = new BlockGenerator(storageLevel)
-
-  protected override def onStart() {
-    val responder = new SpecificResponder(
-      classOf[AvroSourceProtocol], new FlumeEventServer(this))
-    val server = new NettyServer(responder, new InetSocketAddress(host, port))
-    blockGenerator.start()
-    server.start()
-    logInfo("Flume receiver started")
-  }
-
-  protected override def onStop() {
-    blockGenerator.stop()
-    logInfo("Flume receiver stopped")
-  }
-
-  override def getLocationPreference = Some(host)
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
deleted file mode 100644
index 526f556..0000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.dstream
-
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.StreamingContext
-
-import java.util.Properties
-import java.util.concurrent.Executors
-
-import kafka.consumer._
-import kafka.serializer.Decoder
-import kafka.utils.VerifiableProperties
-import kafka.utils.ZKStringSerializer
-import org.I0Itec.zkclient._
-
-import scala.collection.Map
-import scala.reflect.ClassTag
-
-/**
- * Input stream that pulls messages from a Kafka Broker.
- *
- * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- * @param storageLevel RDD storage level.
- */
-private[streaming]
-class KafkaInputDStream[
-  K: ClassTag,
-  V: ClassTag,
-  U <: Decoder[_]: Manifest,
-  T <: Decoder[_]: Manifest](
-    @transient ssc_ : StreamingContext,
-    kafkaParams: Map[String, String],
-    topics: Map[String, Int],
-    storageLevel: StorageLevel
-  ) extends NetworkInputDStream[(K, V)](ssc_) with Logging {
-
-  def getReceiver(): NetworkReceiver[(K, V)] = {
-    new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)
-        .asInstanceOf[NetworkReceiver[(K, V)]]
-  }
-}
-
-private[streaming]
-class KafkaReceiver[
-  K: ClassTag,
-  V: ClassTag,
-  U <: Decoder[_]: Manifest,
-  T <: Decoder[_]: Manifest](
-    kafkaParams: Map[String, String],
-    topics: Map[String, Int],
-    storageLevel: StorageLevel
-  ) extends NetworkReceiver[Any] {
-
-  // Handles pushing data into the BlockManager
-  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
-  // Connection to Kafka
-  var consumerConnector : ConsumerConnector = null
-
-  def onStop() {
-    blockGenerator.stop()
-  }
-
-  def onStart() {
-
-    blockGenerator.start()
-
-    // In case we are using multiple Threads to handle Kafka Messages
-    val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
-
-    logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id"))
-
-    // Kafka connection properties
-    val props = new Properties()
-    kafkaParams.foreach(param => props.put(param._1, param._2))
-
-    // Create the connection to the cluster
-    logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect"))
-    val consumerConfig = new ConsumerConfig(props)
-    consumerConnector = Consumer.create(consumerConfig)
-    logInfo("Connected to " + kafkaParams("zookeeper.connect"))
-
-    // When autooffset.reset is defined, it is our responsibility to try and whack the
-    // consumer group zk node.
-    if (kafkaParams.contains("auto.offset.reset")) {
-      tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id"))
-    }
-
-    val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
-      .newInstance(consumerConfig.props)
-      .asInstanceOf[Decoder[K]]
-    val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
-      .newInstance(consumerConfig.props)
-      .asInstanceOf[Decoder[V]]
-
-    // Create Threads for each Topic/Message Stream we are listening
-    val topicMessageStreams = consumerConnector.createMessageStreams(
-      topics, keyDecoder, valueDecoder)
-
-
-    // Start the messages handler for each partition
-    topicMessageStreams.values.foreach { streams =>
-      streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
-    }
-  }
-
-  // Handles Kafka Messages
-  private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V])
-    extends Runnable {
-    def run() {
-      logInfo("Starting MessageHandler.")
-      for (msgAndMetadata <- stream) {
-        blockGenerator += (msgAndMetadata.key, msgAndMetadata.message)
-      }
-    }
-  }
-
-  // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because
-  // Kafka 0.7.2 only honors this param when the group is not in zookeeper.
-  //
-  // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas'
-  // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest':
-  // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
-  private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) {
-    try {
-      val dir = "/consumers/" + groupId
-      logInfo("Cleaning up temporary zookeeper data under " + dir + ".")
-      val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer)
-      zk.deleteRecursive(dir)
-      zk.close()
-    } catch {
-      case _ : Throwable => // swallow
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
deleted file mode 100644
index ef4a737..0000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.dstream
-
-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
-import java.io.IOException
-
-import org.eclipse.paho.client.mqttv3.MqttCallback
-import org.eclipse.paho.client.mqttv3.MqttClient
-import org.eclipse.paho.client.mqttv3.MqttClientPersistence
-import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence
-import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken
-import org.eclipse.paho.client.mqttv3.MqttException
-import org.eclipse.paho.client.mqttv3.MqttMessage
-import org.eclipse.paho.client.mqttv3.MqttTopic
-
-import scala.collection.Map
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
-import scala.reflect.ClassTag
-
-/**
- * Input stream that subscribe messages from a Mqtt Broker.
- * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/
- * @param brokerUrl Url of remote mqtt publisher
- * @param topic topic name to subscribe to
- * @param storageLevel RDD storage level.
- */
-
-private[streaming] 
-class MQTTInputDStream[T: ClassTag](
-  @transient ssc_ : StreamingContext,
-  brokerUrl: String,
-  topic: String,
-  storageLevel: StorageLevel
-  ) extends NetworkInputDStream[T](ssc_) with Logging {
-  
-  def getReceiver(): NetworkReceiver[T] = {
-    new MQTTReceiver(brokerUrl, topic, storageLevel)
-      .asInstanceOf[NetworkReceiver[T]]
-  }
-}
-
-private[streaming] 
-class MQTTReceiver(brokerUrl: String,
-  topic: String,
-  storageLevel: StorageLevel
-  ) extends NetworkReceiver[Any] {
-  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
-  
-  def onStop() {
-    blockGenerator.stop()
-  }
-  
-  def onStart() {
-
-    blockGenerator.start()
-
-    // Set up persistence for messages 
-    var peristance: MqttClientPersistence = new MemoryPersistence()
-
-    // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance
-    var client: MqttClient = new MqttClient(brokerUrl, "MQTTSub", peristance)
-
-    // Connect to MqttBroker    
-    client.connect()
-
-    // Subscribe to Mqtt topic
-    client.subscribe(topic)
-
-    // Callback automatically triggers as and when new message arrives on specified topic
-    var callback: MqttCallback = new MqttCallback() {
-
-      // Handles Mqtt message 
-      override def messageArrived(arg0: String, arg1: MqttMessage) {
-        blockGenerator += new String(arg1.getPayload())
-      }
-
-      override def deliveryComplete(arg0: IMqttDeliveryToken) {
-      }
-
-      override def connectionLost(arg0: Throwable) {
-        logInfo("Connection lost " + arg0)
-      }
-    }
-
-    // Set up callback for MqttClient
-    client.setCallback(callback)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
deleted file mode 100644
index f164d51..0000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.receivers
-
-import scala.reflect.ClassTag
-
-import akka.actor.Actor
-import akka.util.ByteString
-import akka.zeromq._
-
-import org.apache.spark.Logging
-
-/**
- * A receiver to subscribe to ZeroMQ stream.
- */
-private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String,
-  subscribe: Subscribe,
-  bytesToObjects: Seq[ByteString] ⇒ Iterator[T])
-  extends Actor with Receiver with Logging {
-
-  override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self),
-    Connect(publisherUrl), subscribe)
-
-  def receive: Receive = {
-
-    case Connecting ⇒ logInfo("connecting ...")
-
-    case m: ZMQMessage ⇒
-      logDebug("Received message for:" + m.frame(0))
-
-      //We ignore first frame for processing as it is the topic
-      val bytes = m.frames.tail
-      pushBlock(bytesToObjects(bytes))
-
-    case Closed ⇒ logInfo("received closed ")
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
----------------------------------------------------------------------
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 1cd0b9b..2734393 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
@@ -33,6 +33,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
   val ssc = jobScheduler.ssc
   val clockClass = System.getProperty(
     "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
+  logInfo("Using clock class = " + clockClass)
   val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => generateJobs(new Time(longTime)))

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
index abff55d..4a8e15d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
@@ -160,7 +160,10 @@ class NetworkInputTracker(
       }
       // Run the dummy Spark job to ensure that all slaves have registered.
       // This avoids all the receivers to be scheduled on the same node.
-      ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+      if (!ssc.sparkContext.isLocal) {
+        ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+      }
+
 
       // Distribute the receivers and start them
       ssc.sparkContext.runJob(tempRDD, startReceiver)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
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 daeb99f..f4d26c0 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -17,22 +17,16 @@
 
 package org.apache.spark.streaming;
 
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-
-import kafka.serializer.StringDecoder;
+import scala.Tuple2;
 
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
-import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
+import java.io.*;
+import java.util.*;
 
-import scala.Tuple2;
-import twitter4j.Status;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
 
 import org.apache.spark.HashPartitioner;
 import org.apache.spark.api.java.JavaPairRDD;
@@ -43,39 +37,11 @@ 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.dstream.SparkFlumeEvent;
-import org.apache.spark.streaming.JavaTestUtils;
-import org.apache.spark.streaming.JavaCheckpointTestUtils;
-
-import java.io.*;
-import java.util.*;
-
-import akka.actor.Props;
-import akka.zeromq.Subscribe;
-
 
 // The test suite itself is Serializable so that anonymous Function implementations can be
 // serialized, as an alternative to converting these anonymous classes to static inner classes;
 // see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
-  private transient JavaStreamingContext ssc;
-
-  @Before
-  public void setUp() {
-      System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
-      ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
-    ssc.checkpoint("checkpoint");
-  }
-
-  @After
-  public void tearDown() {
-    ssc.stop();
-    ssc = null;
-
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.driver.port");
-  }
-
+public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable {
   @Test
   public void testCount() {
     List<List<Integer>> inputData = Arrays.asList(
@@ -1597,26 +1563,6 @@ public class JavaAPISuite implements Serializable {
   // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
   // InputStream functionality is deferred to the existing Scala tests.
   @Test
-  public void testKafkaStream() {
-    HashMap<String, Integer> topics = Maps.newHashMap();
-    JavaPairDStream<String, String> test1 = ssc.kafkaStream("localhost:12345", "group", topics);
-    JavaPairDStream<String, String> test2 = ssc.kafkaStream("localhost:12345", "group", topics,
-      StorageLevel.MEMORY_AND_DISK());
-
-    HashMap<String, String> kafkaParams = Maps.newHashMap();
-    kafkaParams.put("zookeeper.connect","localhost:12345");
-    kafkaParams.put("group.id","consumer-group");
-    JavaPairDStream<String, String> test3 = ssc.kafkaStream(
-      String.class,
-      String.class,
-      StringDecoder.class,
-      StringDecoder.class,
-      kafkaParams,
-      topics,
-      StorageLevel.MEMORY_AND_DISK());
-  }
-
-  @Test
   public void testSocketTextStream() {
     JavaDStream<String> test = ssc.socketTextStream("localhost", 12345);
   }
@@ -1654,16 +1600,10 @@ public class JavaAPISuite implements Serializable {
   public void testRawSocketStream() {
     JavaDStream<String> test = ssc.rawSocketStream("localhost", 12345);
   }
-
-  @Test
-  public void testFlumeStream() {
-    JavaDStream<SparkFlumeEvent> test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
-  }
-
+  /*
   @Test
   public void testFileStream() {
-    JavaPairDStream<String, String> foo =
-      ssc.<String, String, SequenceFileInputFormat<String,String>>fileStream("/tmp/foo");
+    JavaPairDStream<String, String> foo = ssc.<String, String, SequenceFileInputFormat<String,String>>fileStream("/tmp/foo");
   }
 
   @Test
@@ -1685,5 +1625,5 @@ public class JavaAPISuite implements Serializable {
         return null;
       }
     });
-  }
+  } */
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000..34bee56
--- /dev/null
+++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+    protected transient JavaStreamingContext ssc;
+
+    @Before
+    public void setUp() {
+        System.clearProperty("spark.driver.port");
+        System.clearProperty("spark.hostPort");
+        System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+        ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+        ssc.checkpoint("checkpoint");
+    }
+
+    @After
+    public void tearDown() {
+        ssc.stop();
+        ssc = null;
+
+        // 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");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
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 62a9f12..0cffed6 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -23,7 +23,7 @@ import akka.actor.IOManager
 import akka.actor.Props
 import akka.util.ByteString
 
-import org.apache.spark.streaming.dstream.{NetworkReceiver, SparkFlumeEvent}
+import org.apache.spark.streaming.dstream.{NetworkReceiver}
 import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
 import java.io.{File, BufferedWriter, OutputStreamWriter}
 import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue}
@@ -31,18 +31,11 @@ import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
 import util.ManualClock
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.receivers.Receiver
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.Logging
 import scala.util.Random
 import org.apache.commons.io.FileUtils
 import org.scalatest.BeforeAndAfter
-import org.apache.flume.source.avro.AvroSourceProtocol
-import org.apache.flume.source.avro.AvroFlumeEvent
-import org.apache.flume.source.avro.Status
-import org.apache.avro.ipc.{specific, NettyTransceiver}
-import org.apache.avro.ipc.specific.SpecificRequestor
-import java.nio.ByteBuffer
 import collection.JavaConversions._
-import java.nio.charset.Charset
 import com.google.common.io.Files
 import java.util.concurrent.atomic.AtomicInteger
 
@@ -99,55 +92,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
   }
 
 
-  test("flume input stream") {
-    // Set up the streaming context and input streams
-    val ssc = new StreamingContext(master, framework, batchDuration)
-    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
-    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
-      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
-    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
-    ssc.registerOutputStream(outputStream)
-    ssc.start()
-
-    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    val input = Seq(1, 2, 3, 4, 5)
-    Thread.sleep(1000)
-    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
-    val client = SpecificRequestor.getClient(
-      classOf[AvroSourceProtocol], transceiver)
-
-    for (i <- 0 until input.size) {
-      val event = new AvroFlumeEvent
-      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
-      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
-      client.append(event)
-      Thread.sleep(500)
-      clock.addToTime(batchDuration.milliseconds)
-    }
-
-    val startTime = System.currentTimeMillis()
-    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
-      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
-      Thread.sleep(100)
-    }
-    Thread.sleep(1000)
-    val timeTaken = System.currentTimeMillis() - startTime
-    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
-    logInfo("Stopping context")
-    ssc.stop()
-
-    val decoder = Charset.forName("UTF-8").newDecoder()
-
-    assert(outputBuffer.size === input.length)
-    for (i <- 0 until outputBuffer.size) {
-      assert(outputBuffer(i).size === 1)
-      val str = decoder.decode(outputBuffer(i).head.event.getBody)
-      assert(str.toString === input(i).toString)
-      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
-    }
-  }
-
-
   test("file input stream") {
     // Disable manual clock as FileInputDStream does not work with manual clock
     System.clearProperty("spark.streaming.clock")
@@ -249,21 +193,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     }
   }
 
-  test("kafka input stream") {
-    val ssc = new StreamingContext(master, framework, batchDuration)
-    val topics = Map("my-topic" -> 1)
-    val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
-    val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK)
-
-    // Test specifying decoder
-    val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
-    val test3 = ssc.kafkaStream[
-      String,
-      String,
-      kafka.serializer.StringDecoder,
-      kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
-  }
-
   test("multi-thread receiver") {
     // set up the test receiver
     val numThreads = 10

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f4e40661/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
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 e969e91..f56c046 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -137,11 +137,10 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
   // if you want to add your stuff to "before" (i.e., don't call before { } )
   def beforeFunction() {
     if (useManualClock) {
-      System.setProperty(
-        "spark.streaming.clock",
-        "org.apache.spark.streaming.util.ManualClock"
-      )
+      logInfo("Using manual clock")
+      System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
     } else {
+      logInfo("Using real clock")
       System.clearProperty("spark.streaming.clock")
     }
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
@@ -273,7 +272,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
       val startTime = System.currentTimeMillis()
       while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
         logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
-        Thread.sleep(100)
+        Thread.sleep(10)
       }
       val timeTaken = System.currentTimeMillis() - startTime
 


[04/13] git commit: Added pom.xml for external projects and removed unnecessary dependencies and repositoris from other poms and sbt.

Posted by pw...@apache.org.
Added pom.xml for external projects and removed unnecessary dependencies and repositoris from other poms and sbt.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/97630849
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/97630849
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/97630849

Branch: refs/heads/master
Commit: 97630849ff2cdaa2ff8a115c3e8e6ca8dba7477d
Parents: f4e4066
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Dec 31 00:28:57 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Dec 31 00:28:57 2013 -0800

----------------------------------------------------------------------
 examples/pom.xml         |  45 ++++++++++++------
 external/flume/pom.xml   |  93 ++++++++++++++++++++++++++++++++++++
 external/kafka/pom.xml   |  97 +++++++++++++++++++++++++++++++++++++
 external/mqtt/pom.xml    | 108 ++++++++++++++++++++++++++++++++++++++++++
 external/twitter/pom.xml |  89 ++++++++++++++++++++++++++++++++++
 external/zeromq/pom.xml  |  89 ++++++++++++++++++++++++++++++++++
 pom.xml                  |  29 +++---------
 project/SparkBuild.scala |  21 +++-----
 streaming/pom.xml        |  83 ++++++++++++--------------------
 9 files changed, 548 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 7a7032c..1839667 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -72,6 +72,36 @@
       <scope>provided</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-twitter_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-kafka_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-zeromq_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase</artifactId>
       <version>0.94.6</version>
@@ -87,21 +117,6 @@
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>com.sksamuel.kafka</groupId>
-      <artifactId>kafka_${scala.binary.version}</artifactId>
-      <version>0.8.0-beta1</version>
-      <exclusions>
-        <exclusion>
-          <groupId>com.sun.jmx</groupId>
-          <artifactId>jmxri</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jdmk</groupId>
-          <artifactId>jmxtools</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/external/flume/pom.xml
----------------------------------------------------------------------
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
new file mode 100644
index 0000000..443910a
--- /dev/null
+++ b/external/flume/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-flume_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Flume</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+      <version>1.2.0</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.xerial.snappy</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/external/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
new file mode 100644
index 0000000..f782e0e
--- /dev/null
+++ b/external/kafka/pom.xml
@@ -0,0 +1,97 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-kafka_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Kafka</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sksamuel.kafka</groupId>
+      <artifactId>kafka_${scala.binary.version}</artifactId>
+      <version>0.8.0-beta1</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.sf.jopt-simple</groupId>
+          <artifactId>jopt-simple</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/external/mqtt/pom.xml
----------------------------------------------------------------------
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
new file mode 100644
index 0000000..31b4fa8
--- /dev/null
+++ b/external/mqtt/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-mqtt_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External MQTT</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <repositories>
+    <repository>
+      <id>mqtt-repo</id>
+      <name>MQTT Repository</name>
+      <url>https://repo.eclipse.org/content/repositories/paho-releases</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
+  </repositories>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.paho</groupId>
+      <artifactId>mqtt-client</artifactId>
+       <version>0.4.0</version>
+    </dependency>
+    <dependency>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+      <version>${akka.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/external/twitter/pom.xml
----------------------------------------------------------------------
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
new file mode 100644
index 0000000..216e6c1
--- /dev/null
+++ b/external/twitter/pom.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-twitter_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External Twitter</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.twitter4j</groupId>
+      <artifactId>twitter4j-stream</artifactId>
+      <version>3.0.3</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/external/zeromq/pom.xml
----------------------------------------------------------------------
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
new file mode 100644
index 0000000..c240d59
--- /dev/null
+++ b/external/zeromq/pom.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.spark</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.9.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.apache.spark</groupId>
+  <artifactId>spark-streaming-zeromq_2.10</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project External ZeroMQ</name>
+  <url>http://spark.incubator.apache.org/</url>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>${akka.group}</groupId>
+      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+      <version>${akka.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.jboss.netty</groupId>
+          <artifactId>netty</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.binary.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 57e8435..0257a08 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,12 +87,17 @@
   <modules>
     <module>core</module>
     <module>bagel</module>
-    <module>examples</module>
     <module>mllib</module>
     <module>tools</module>
     <module>streaming</module>
     <module>repl</module>
     <module>assembly</module>
+    <module>external/twitter</module>
+    <module>external/kafka</module>
+    <module>external/flume</module>
+    <module>external/zeromq</module>
+    <module>external/mqtt</module>
+    <module>examples</module>
   </modules>
 
   <properties>
@@ -140,17 +145,6 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
-    <repository>
-      <id>mqtt-repo</id>
-      <name>MQTT Repository</name>
-      <url>https://repo.eclipse.org/content/repositories/paho-releases</url>
-      <releases>
-        <enabled>true</enabled>
-      </releases>
-      <snapshots>
-        <enabled>false</enabled>
-      </snapshots>
-    </repository>
   </repositories>
 
   <dependencyManagement>
@@ -254,17 +248,6 @@
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>${akka.group}</groupId>
-        <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
-        <version>${akka.version}</version>
-        <exclusions>
-          <exclusion>
-            <groupId>org.jboss.netty</groupId>
-            <artifactId>netty</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
         <groupId>it.unimi.dsi</groupId>
         <artifactId>fastutil</artifactId>
         <version>6.4.4</version>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 4e92d2a..b7aa732 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -141,7 +141,7 @@ object SparkBuild extends Build {
     // also check the local Maven repository ~/.m2
     resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))),
 
-   // For Sonatype publishing
+    // For Sonatype publishing
     resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
       "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
 
@@ -315,9 +315,6 @@ object SparkBuild extends Build {
 
   def streamingSettings = sharedSettings ++ Seq(
     name := "spark-streaming",
-    resolvers ++= Seq(
-      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
-    ),
     libraryDependencies ++= Seq(
       "commons-io" % "commons-io" % "2.4" 
     )
@@ -360,14 +357,14 @@ object SparkBuild extends Build {
     }
   )
 
-  def twitterSettings() = streamingSettings ++ Seq(
+  def twitterSettings() = sharedSettings ++ Seq(
     name := "spark-streaming-twitter",
     libraryDependencies ++= Seq(
       "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty)
     )
   )
   
-  def kafkaSettings() = streamingSettings ++ Seq(
+  def kafkaSettings() = sharedSettings ++ Seq(
     name := "spark-streaming-kafka",
     libraryDependencies ++= Seq(
       "com.github.sgroschupf"    % "zkclient"   % "0.1"          excludeAll(excludeNetty),
@@ -379,14 +376,14 @@ object SparkBuild extends Build {
     )
   )
   
-  def flumeSettings() = streamingSettings ++ Seq(
+  def flumeSettings() = sharedSettings ++ Seq(
     name := "spark-streaming-flume",
     libraryDependencies ++= Seq(
       "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy)
     )
   )
 
-  def zeromqSettings() = streamingSettings ++ Seq(
+  def zeromqSettings() = sharedSettings ++ Seq(
     name := "spark-streaming-zeromq",
     libraryDependencies ++= Seq(
       "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
@@ -395,11 +392,7 @@ object SparkBuild extends Build {
 
   def mqttSettings() = streamingSettings ++ Seq(
     name := "spark-streaming-mqtt",
-    resolvers ++= Seq(
-      "Apache repo" at "https://repository.apache.org/content/repositories/releases"
-    ),
-    libraryDependencies ++= Seq(
-      "org.eclipse.paho" % "mqtt-client" % "0.4.0"
-    )
+    resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
+    libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0")
   )
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/97630849/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index e3b6fee..4597569 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -61,59 +61,10 @@
       <version>1.9.11</version>
     </dependency>
     <dependency>
-      <groupId>com.sksamuel.kafka</groupId>
-      <artifactId>kafka_${scala.binary.version}</artifactId>
-      <version>0.8.0-beta1</version>
-      <exclusions>
-        <exclusion>
-          <groupId>com.sun.jmx</groupId>
-          <artifactId>jmxri</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>com.sun.jdmk</groupId>
-          <artifactId>jmxtools</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>net.sf.jopt-simple</groupId>
-          <artifactId>jopt-simple</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.flume</groupId>
-      <artifactId>flume-ng-sdk</artifactId>
-      <version>1.2.0</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.xerial.snappy</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.twitter4j</groupId>
-      <artifactId>twitter4j-stream</artifactId>
-      <version>3.0.3</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.jboss.netty</groupId>
-          <artifactId>netty</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>org.scala-lang</groupId>
       <artifactId>scala-library</artifactId>
     </dependency>
     <dependency>
-      <groupId>${akka.group}</groupId>
-      <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
       <scope>test</scope>
@@ -137,11 +88,6 @@
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.eclipse.paho</groupId>
-      <artifactId>mqtt-client</artifactId>
-       <version>0.4.0</version>
-    </dependency>
   </dependencies>
   <build>
     <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
@@ -151,6 +97,35 @@
         <groupId>org.scalatest</groupId>
         <artifactId>scalatest-maven-plugin</artifactId>
       </plugin>
+      
+      <!-- 
+           This plugin forces the generation of jar containing streaming test classes, 
+           so that the tests classes of external modules can use them. The two execution profiles
+           are necessary - first one for 'mvn package', second one for 'mvn compile'. Ideally, 
+           'mvn compile' should not compile test classes and therefore should not need this. 
+           However, an open Maven bug (http://jira.codehaus.org/browse/MNG-3559)
+           causes the compilation to fail if streaming test-jar is not generated. Hence, the 
+           second execution profile for 'mvn compile'.
+      -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>test-jar-on-compile</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>


[06/13] git commit: Removed extra empty lines.

Posted by pw...@apache.org.
Removed extra empty lines.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/87b915f2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/87b915f2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/87b915f2

Branch: refs/heads/master
Commit: 87b915f22105ced8b9cad2a1262a0fd26542ee4f
Parents: 3ab297a
Author: Tathagata Das <ta...@gmail.com>
Authored: Tue Dec 31 00:42:10 2013 -0800
Committer: Tathagata Das <ta...@gmail.com>
Committed: Tue Dec 31 00:42:10 2013 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala  | 1 -
 .../org/apache/spark/streaming/twitter/TwitterInputDStream.scala    | 1 -
 .../org/apache/spark/streaming/scheduler/NetworkInputTracker.scala  | 1 -
 3 files changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87b915f2/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
index fd69328..a2cd49c 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -34,7 +34,6 @@ import org.apache.spark.storage.StorageLevel
 import org.apache.spark.streaming.StreamingContext
 import org.apache.spark.streaming.dstream._
 
-
 /**
  * Input stream that pulls messages from a Kafka Broker.
  *

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87b915f2/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
----------------------------------------------------------------------
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
index 97e48eb..5cc721d 100644
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
@@ -98,4 +98,3 @@ class TwitterReceiver(
     logInfo("Twitter receiver stopped")
   }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87b915f2/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
index 4a8e15d..75f7244 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
@@ -164,7 +164,6 @@ class NetworkInputTracker(
         ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
       }
 
-
       // Distribute the receivers and start them
       ssc.sparkContext.runJob(tempRDD, startReceiver)
     }