You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bahir.apache.org by lr...@apache.org on 2016/06/10 15:24:02 UTC

[23/50] [abbrv] bahir git commit: [SPARK-12510][STREAMING] Refactor ActorReceiver to support Java

[SPARK-12510][STREAMING] Refactor ActorReceiver to support Java

This PR includes the following changes:

1. Rename `ActorReceiver` to `ActorReceiverSupervisor`
2. Remove `ActorHelper`
3. Add a new `ActorReceiver` for Scala and `JavaActorReceiver` for Java
4. Add `JavaActorWordCount` example

Author: Shixiong Zhu <sh...@databricks.com>

Closes #10457 from zsxwing/java-actor-stream.


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

Branch: refs/heads/master
Commit: cca0efef68e91fd6d30cbae0a73fe1c737cd28a1
Parents: 67f0c90
Author: Shixiong Zhu <sh...@databricks.com>
Authored: Thu Jan 7 15:26:55 2016 -0800
Committer: Shixiong Zhu <sh...@databricks.com>
Committed: Thu Jan 7 15:26:55 2016 -0800

----------------------------------------------------------------------
 .../streaming/akka/JavaActorWordCount.java      | 137 +++++++++++++++++++
 .../streaming/akka/ActorWordCount.scala         |   9 +-
 .../spark/streaming/akka/ActorReceiver.scala    |  64 +++++++--
 .../spark/streaming/zeromq/ZeroMQReceiver.scala |   5 +-
 4 files changed, 197 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bahir/blob/cca0efef/streaming-akka/examples/src/main/java/org/apache/spark/examples/streaming/akka/JavaActorWordCount.java
----------------------------------------------------------------------
diff --git a/streaming-akka/examples/src/main/java/org/apache/spark/examples/streaming/akka/JavaActorWordCount.java b/streaming-akka/examples/src/main/java/org/apache/spark/examples/streaming/akka/JavaActorWordCount.java
new file mode 100644
index 0000000..2377207
--- /dev/null
+++ b/streaming-akka/examples/src/main/java/org/apache/spark/examples/streaming/akka/JavaActorWordCount.java
@@ -0,0 +1,137 @@
+/*
+ * 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.examples.streaming;
+
+import java.util.Arrays;
+
+import scala.Tuple2;
+
+import akka.actor.ActorSelection;
+import akka.actor.Props;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.receiver.JavaActorReceiver;
+
+/**
+ * A sample actor as receiver, is also simplest. This receiver actor
+ * goes and subscribe to a typical publisher/feeder actor and receives
+ * data.
+ *
+ * @see [[org.apache.spark.examples.streaming.FeederActor]]
+ */
+class JavaSampleActorReceiver<T> extends JavaActorReceiver {
+
+  private final String urlOfPublisher;
+
+  public JavaSampleActorReceiver(String urlOfPublisher) {
+    this.urlOfPublisher = urlOfPublisher;
+  }
+
+  private ActorSelection remotePublisher;
+
+  @Override
+  public void preStart() {
+    remotePublisher = getContext().actorSelection(urlOfPublisher);
+    remotePublisher.tell(new SubscribeReceiver(getSelf()), getSelf());
+  }
+
+  public void onReceive(Object msg) throws Exception {
+    store((T) msg);
+  }
+
+  @Override
+  public void postStop() {
+    remotePublisher.tell(new UnsubscribeReceiver(getSelf()), getSelf());
+  }
+}
+
+/**
+ * A sample word count program demonstrating the use of plugging in
+ * Actor as Receiver
+ * Usage: JavaActorWordCount <hostname> <port>
+ *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
+ *
+ * To run this example locally, you may run Feeder Actor as
+ * <code><pre>
+ *     $ bin/run-example org.apache.spark.examples.streaming.FeederActor localhost 9999
+ * </pre></code>
+ * and then run the example
+ * <code><pre>
+ *     $ bin/run-example org.apache.spark.examples.streaming.JavaActorWordCount localhost 9999
+ * </pre></code>
+ */
+public class JavaActorWordCount {
+
+  public static void main(String[] args) {
+    if (args.length < 2) {
+      System.err.println("Usage: JavaActorWordCount <hostname> <port>");
+      System.exit(1);
+    }
+
+    StreamingExamples.setStreamingLogLevels();
+
+    final String host = args[0];
+    final String port = args[1];
+    SparkConf sparkConf = new SparkConf().setAppName("JavaActorWordCount");
+    // Create the context and set the batch size
+    JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000));
+
+    String feederActorURI = "akka.tcp://test@" + host + ":" + port + "/user/FeederActor";
+
+    /*
+     * Following is the use of actorStream to plug in custom actor as receiver
+     *
+     * An important point to note:
+     * Since Actor may exist outside the spark framework, It is thus user's responsibility
+     * to ensure the type safety, i.e type of data received and InputDstream
+     * should be same.
+     *
+     * For example: Both actorStream and JavaSampleActorReceiver are parameterized
+     * to same type to ensure type safety.
+     */
+    JavaDStream<String> lines = jssc.actorStream(
+        Props.create(JavaSampleActorReceiver.class, feederActorURI), "SampleReceiver");
+
+    // compute wordcount
+    lines.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterable<String> call(String s) {
+        return Arrays.asList(s.split("\\s+"));
+      }
+    }).mapToPair(new PairFunction<String, String, Integer>() {
+      @Override
+      public Tuple2<String, Integer> call(String s) {
+        return new Tuple2<String, Integer>(s, 1);
+      }
+    }).reduceByKey(new Function2<Integer, Integer, Integer>() {
+      @Override
+      public Integer call(Integer i1, Integer i2) {
+        return i1 + i2;
+      }
+    }).print();
+
+    jssc.start();
+    jssc.awaitTermination();
+  }
+}

http://git-wip-us.apache.org/repos/asf/bahir/blob/cca0efef/streaming-akka/examples/src/main/scala/org/apache/spark/examples/streaming/akka/ActorWordCount.scala
----------------------------------------------------------------------
diff --git a/streaming-akka/examples/src/main/scala/org/apache/spark/examples/streaming/akka/ActorWordCount.scala b/streaming-akka/examples/src/main/scala/org/apache/spark/examples/streaming/akka/ActorWordCount.scala
index a47fb7b..88cdc6b 100644
--- a/streaming-akka/examples/src/main/scala/org/apache/spark/examples/streaming/akka/ActorWordCount.scala
+++ b/streaming-akka/examples/src/main/scala/org/apache/spark/examples/streaming/akka/ActorWordCount.scala
@@ -26,8 +26,7 @@ import akka.actor.{actorRef2Scala, Actor, ActorRef, Props}
 
 import org.apache.spark.{SecurityManager, SparkConf}
 import org.apache.spark.streaming.{Seconds, StreamingContext}
-import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
-import org.apache.spark.streaming.receiver.ActorHelper
+import org.apache.spark.streaming.receiver.ActorReceiver
 import org.apache.spark.util.AkkaUtils
 
 case class SubscribeReceiver(receiverActor: ActorRef)
@@ -80,7 +79,7 @@ class FeederActor extends Actor {
  * @see [[org.apache.spark.examples.streaming.FeederActor]]
  */
 class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
-extends Actor with ActorHelper {
+extends ActorReceiver {
 
   lazy private val remotePublisher = context.actorSelection(urlOfPublisher)
 
@@ -127,9 +126,9 @@ object FeederActor {
  *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
  *
  * To run this example locally, you may run Feeder Actor as
- *    `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.0.1 9999`
+ *    `$ bin/run-example org.apache.spark.examples.streaming.FeederActor localhost 9999`
  * and then run the example
- *    `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.0.1 9999`
+ *    `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount localhost 9999`
  */
 object ActorWordCount {
   def main(args: Array[String]) {

http://git-wip-us.apache.org/repos/asf/bahir/blob/cca0efef/streaming-akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala
----------------------------------------------------------------------
diff --git a/streaming-akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala b/streaming-akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala
index 7ec7401..0eabf3d 100644
--- a/streaming-akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala
+++ b/streaming-akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala
@@ -47,13 +47,12 @@ object ActorSupervisorStrategy {
 
 /**
  * :: DeveloperApi ::
- * A receiver trait to be mixed in with your Actor to gain access to
- * the API for pushing received data into Spark Streaming for being processed.
+ * A base Actor that provides APIs for pushing received data into Spark Streaming for processing.
  *
  * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html
  *
  * @example {{{
- *  class MyActor extends Actor with ActorHelper{
+ *  class MyActor extends ActorReceiver {
  *      def receive {
  *          case anything: String => store(anything)
  *      }
@@ -69,13 +68,60 @@ object ActorSupervisorStrategy {
  *       should be same.
  */
 @DeveloperApi
-trait ActorHelper extends Logging{
+abstract class ActorReceiver extends Actor {
 
-  self: Actor => // to ensure that this can be added to Actor classes only
+  /** Store an iterator of received data as a data block into Spark's memory. */
+  def store[T](iter: Iterator[T]) {
+    context.parent ! IteratorData(iter)
+  }
+
+  /**
+   * Store the bytes of received data as a data block into Spark's memory. Note
+   * that the data in the ByteBuffer must be serialized using the same serializer
+   * that Spark is configured to use.
+   */
+  def store(bytes: ByteBuffer) {
+    context.parent ! ByteBufferData(bytes)
+  }
+
+  /**
+   * Store a single item of received data to Spark's memory.
+   * These single items will be aggregated together into data blocks before
+   * being pushed into Spark's memory.
+   */
+  def store[T](item: T) {
+    context.parent ! SingleItemData(item)
+  }
+}
+
+/**
+ * :: DeveloperApi ::
+ * A Java UntypedActor that provides APIs for pushing received data into Spark Streaming for
+ * processing.
+ *
+ * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html
+ *
+ * @example {{{
+ *  class MyActor extends JavaActorReceiver {
+ *      def receive {
+ *          case anything: String => store(anything)
+ *      }
+ *  }
+ *
+ *  // Can be used with an actorStream as follows
+ *  ssc.actorStream[String](Props(new MyActor),"MyActorReceiver")
+ *
+ * }}}
+ *
+ * @note Since Actor may exist outside the spark framework, It is thus user's responsibility
+ *       to ensure the type safety, i.e parametrized type of push block and InputDStream
+ *       should be same.
+ */
+@DeveloperApi
+abstract class JavaActorReceiver extends UntypedActor {
 
   /** Store an iterator of received data as a data block into Spark's memory. */
   def store[T](iter: Iterator[T]) {
-    logDebug("Storing iterator")
     context.parent ! IteratorData(iter)
   }
 
@@ -85,7 +131,6 @@ trait ActorHelper extends Logging{
    * that Spark is configured to use.
    */
   def store(bytes: ByteBuffer) {
-    logDebug("Storing Bytes")
     context.parent ! ByteBufferData(bytes)
   }
 
@@ -95,7 +140,6 @@ trait ActorHelper extends Logging{
    * being pushed into Spark's memory.
    */
   def store[T](item: T) {
-    logDebug("Storing item")
     context.parent ! SingleItemData(item)
   }
 }
@@ -104,7 +148,7 @@ trait ActorHelper extends Logging{
  * :: DeveloperApi ::
  * Statistics for querying the supervisor about state of workers. Used in
  * conjunction with `StreamingContext.actorStream` and
- * [[org.apache.spark.streaming.receiver.ActorHelper]].
+ * [[org.apache.spark.streaming.receiver.ActorReceiver]].
  */
 @DeveloperApi
 case class Statistics(numberOfMsgs: Int,
@@ -137,7 +181,7 @@ private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorRec
  *  context.parent ! Props(new Worker, "Worker")
  * }}}
  */
-private[streaming] class ActorReceiver[T: ClassTag](
+private[streaming] class ActorReceiverSupervisor[T: ClassTag](
     props: Props,
     name: String,
     storageLevel: StorageLevel,

http://git-wip-us.apache.org/repos/asf/bahir/blob/cca0efef/streaming-zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
----------------------------------------------------------------------
diff --git a/streaming-zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/streaming-zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
index 588e6ba..506ba87 100644
--- a/streaming-zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
+++ b/streaming-zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
@@ -19,12 +19,11 @@ 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.receiver.ActorHelper
+import org.apache.spark.streaming.receiver.ActorReceiver
 
 /**
  * A receiver to subscribe to ZeroMQ stream.
@@ -33,7 +32,7 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](
     publisherUrl: String,
     subscribe: Subscribe,
     bytesToObjects: Seq[ByteString] => Iterator[T])
-  extends Actor with ActorHelper with Logging {
+  extends ActorReceiver with Logging {
 
   override def preStart(): Unit = {
     ZeroMQExtension(context.system)