Updated Branches:
  refs/heads/master 5fecd2516 -> 2a05403a7

Made some classes private[stremaing] and deprecated a method in 
JavaStreamingContext.


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

Branch: refs/heads/master
Commit: 9e6375349e945fed87bbeae565f35ab9ca28f70f
Parents: 0e15bd7
Author: Tathagata Das <tathagata.das1...@gmail.com>
Authored: Wed Jan 15 12:15:46 2014 -0800
Committer: Tathagata Das <tathagata.das1...@gmail.com>
Committed: Wed Jan 15 12:15:46 2014 -0800

----------------------------------------------------------------------
 .../apache/spark/streaming/api/java/JavaStreamingContext.scala | 6 +++++-
 .../apache/spark/streaming/util/RateLimitedOutputStream.scala  | 1 +
 .../scala/org/apache/spark/streaming/util/RawTextHelper.scala  | 1 +
 .../scala/org/apache/spark/streaming/util/RawTextSender.scala  | 1 +
 .../src/test/java/org/apache/spark/streaming/JavaAPISuite.java | 6 +++---
 5 files changed, 11 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9e637534/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 4edf8fa..613683c 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
@@ -141,8 +141,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    */
   def this(path: String, hadoopConf: Configuration) = this(new 
StreamingContext(path, hadoopConf))
 
+
+  @deprecated("use sparkContext", "0.9.0")
+  val sc: JavaSparkContext = sparkContext
+
   /** The underlying SparkContext */
-  val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
+  val sparkContext = new JavaSparkContext(ssc.sc)
 
   /**
    * Create a input stream from network source hostname:port. Data is received 
using

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9e637534/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
----------------------------------------------------------------------
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
index b9c0596..179fd75 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
@@ -22,6 +22,7 @@ import scala.annotation.tailrec
 import java.io.OutputStream
 import java.util.concurrent.TimeUnit._
 
+private[streaming]
 class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends 
OutputStream {
   val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS)
   val CHUNK_SIZE = 8192

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9e637534/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
----------------------------------------------------------------------
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
index 5b6c048..07021eb 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
@@ -22,6 +22,7 @@ import org.apache.spark.SparkContext._
 import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
 import scala.collection.JavaConversions.mapAsScalaMap
 
+private[streaming]
 object RawTextHelper {
 
   /** 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9e637534/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
----------------------------------------------------------------------
diff --git 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index 463617a..684b38e 100644
--- 
a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ 
b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -33,6 +33,7 @@ import org.apache.spark.util.IntParam
  * A helper program that sends blocks of Kryo-serialized text strings out on a 
socket at a
  * specified rate. Used to feed data into RawInputDStream.
  */
+private[streaming]
 object RawTextSender extends Logging {
   def main(args: Array[String]) {
     if (args.length != 4) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/9e637534/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 8b7d770..4fbbce9 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -297,9 +297,9 @@ public class JavaAPISuite extends LocalJavaStreamingContext 
implements Serializa
         Arrays.asList(7,8,9));
 
     JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
-    JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1, 2, 3));
-    JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4, 5, 6));
-    JavaRDD<Integer> rdd3 = ssc.sc().parallelize(Arrays.asList(7,8,9));
+    JavaRDD<Integer> rdd1 = ssc.sparkContext().parallelize(Arrays.asList(1, 2, 
3));
+    JavaRDD<Integer> rdd2 = ssc.sparkContext().parallelize(Arrays.asList(4, 5, 
6));
+    JavaRDD<Integer> rdd3 = 
ssc.sparkContext().parallelize(Arrays.asList(7,8,9));
 
     LinkedList<JavaRDD<Integer>> rdds = Lists.newLinkedList();
     rdds.add(rdd1);

Reply via email to