aboutsummaryrefslogtreecommitdiff
path: root/external/flume/src
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2014-01-13 23:23:46 -0800
committerTathagata Das <tathagata.das1565@gmail.com>2014-01-13 23:23:46 -0800
commit4e497db8f3826cf5142b2165a08d02c6f3c2cd90 (patch)
tree9ec25e86ccf8986035215e51f7b0e1ba1b96dad6 /external/flume/src
parent1233b3de01be1ff57910786f5f3e2e2a23e228ab (diff)
downloadspark-4e497db8f3826cf5142b2165a08d02c6f3c2cd90.tar.gz
spark-4e497db8f3826cf5142b2165a08d02c6f3c2cd90.tar.bz2
spark-4e497db8f3826cf5142b2165a08d02c6f3c2cd90.zip
Removed StreamingContext.registerInputStream and registerOutputStream - they were useless as InputDStream has been made to register itself. Also made DStream.register() private[streaming] - not useful to expose the confusing function. Updated a lot of documentation.
Diffstat (limited to 'external/flume/src')
-rw-r--r--external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala1
-rw-r--r--external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala2
2 files changed, 1 insertions, 2 deletions
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
index d53b66dd46..654ba451e7 100644
--- 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
@@ -37,7 +37,6 @@ object FlumeUtils {
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[SparkFlumeEvent] = {
val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
- ssc.registerInputStream(inputStream)
inputStream
}
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 2e8e9fac45..8bc43972ab 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
@@ -43,7 +43,7 @@ class FlumeStreamSuite extends TestSuiteBase {
val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
with SynchronizedBuffer[Seq[SparkFlumeEvent]]
val outputStream = new TestOutputStream(flumeStream, outputBuffer)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]