From c0c397509bc909b9bf2d5186182f461155b021ab Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 7 Jan 2016 15:26:55 -0800 Subject: [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 Closes #10457 from zsxwing/java-actor-stream. --- .../examples/streaming/JavaActorWordCount.java | 137 +++++++++++++++++++++ .../spark/examples/streaming/ActorWordCount.scala | 9 +- .../spark/streaming/zeromq/ZeroMQReceiver.scala | 5 +- project/MimaExcludes.scala | 3 + .../apache/spark/streaming/StreamingContext.scala | 4 +- .../spark/streaming/receiver/ActorReceiver.scala | 64 ++++++++-- 6 files changed, 202 insertions(+), 20 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java new file mode 100644 index 0000000000..2377207779 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/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 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 + * and 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 localhost 9999
+ * 
+ * and then run the example + *
+ *     $ bin/run-example org.apache.spark.examples.streaming.JavaActorWordCount localhost 9999
+ * 
+ */ +public class JavaActorWordCount { + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: JavaActorWordCount "); + 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 lines = jssc.actorStream( + Props.create(JavaSampleActorReceiver.class, feederActorURI), "SampleReceiver"); + + // compute wordcount + lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String s) { + return Arrays.asList(s.split("\\s+")); + } + }).mapToPair(new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }).print(); + + jssc.start(); + jssc.awaitTermination(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index a47fb7b7d7..88cdc6bc14 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/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 { * and 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]) { 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 index 588e6bac7b..506ba8782d 100644 --- 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 @@ -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) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 43ca4690dc..69e5bc881b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -119,6 +119,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$Multiplier"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.Vector$") + ) ++ Seq( + // SPARK-12510 Refactor ActorReceiver to support Java + ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") ) case v if v.startsWith("1.6") => Seq( 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 ca0a21fbb7..ba509a1030 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -41,7 +41,7 @@ import org.apache.spark.serializer.SerializationDebugger import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContextState._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} +import org.apache.spark.streaming.receiver.{ActorReceiverSupervisor, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler.{JobScheduler, StreamingListener} import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.{AsynchronousListenerBus, CallSite, ShutdownHookManager, ThreadUtils, Utils} @@ -312,7 +312,7 @@ class StreamingContext private[streaming] ( storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy ): ReceiverInputDStream[T] = withNamedScope("actor stream") { - receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + receiverStream(new ActorReceiverSupervisor[T](props, name, storageLevel, supervisorStrategy)) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 7ec74016a1..0eabf3d260 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/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, -- cgit v1.2.3