Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 12 additions & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -222,6 +222,18 @@ def contains_file(self, filename):
)


streaming_akka = Module(
name="streaming-akka",
dependencies=[streaming],
source_file_regexes=[
"external/akka",
],
sbt_test_goals=[
"streaming-akka/test",
]
)


streaming_flume = Module(
name="streaming-flume",
dependencies=[streaming],
Expand Down
49 changes: 39 additions & 10 deletions docs/streaming-custom-receivers.md
Original file line number Diff line number Diff line change
Expand Up @@ -257,25 +257,54 @@ The following table summarizes the characteristics of both types of receivers

## Implementing and Using a Custom Actor-based Receiver

<div class="codetabs">
<div data-lang="scala" markdown="1" >

Custom [Akka Actors](http://doc.akka.io/docs/akka/2.3.11/scala/actors.html) can also be used to
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also make sure to add this is not supported in Python?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper)
trait can be mixed in to any Akka actor, which allows received data to be stored in Spark using
`store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc.
receive data. Extending [`ActorReceiver`](api/scala/index.html#org.apache.spark.streaming.akka.ActorReceiver)
allows received data to be stored in Spark using `store(...)` methods. The supervisor strategy of
this actor can be configured to handle failures, etc.

{% highlight scala %}
class CustomActor extends Actor with ActorHelper {

class CustomActor extends ActorReceiver {
def receive = {
case data: String => store(data)
}
}

// A new input stream can be created with this custom actor as
val ssc: StreamingContext = ...
val lines = AkkaUtils.createStream[String](ssc, Props[CustomActor](), "CustomReceiver")

{% endhighlight %}

And a new input stream can be created with this custom actor as
See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala) for an end-to-end example.
</div>
<div data-lang="java" markdown="1">

Custom [Akka UntypedActors](http://doc.akka.io/docs/akka/2.3.11/java/untyped-actors.html) can also be used to
receive data. Extending [`JavaActorReceiver`](api/scala/index.html#org.apache.spark.streaming.akka.JavaActorReceiver)
allows received data to be stored in Spark using `store(...)` methods. The supervisor strategy of
this actor can be configured to handle failures, etc.

{% highlight java %}

class CustomActor extends JavaActorReceiver {
@Override
public void onReceive(Object msg) throws Exception {
store((String) msg);
}
}

// A new input stream can be created with this custom actor as
JavaStreamingContext jssc = ...;
JavaDStream<String> lines = AkkaUtils.<String>createStream(jssc, Props.create(CustomActor.class), "CustomReceiver");

{% highlight scala %}
val ssc: StreamingContext = ...
val lines = ssc.actorStream[String](Props[CustomActor], "CustomReceiver")
{% endhighlight %}

See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala)
for an end-to-end example.
See [JavaActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaActorWordCount.scala) for an end-to-end example.
</div>
</div>

<span class="badge" style="background-color: grey">Python API</span> Since actors are available only in the Java and Scala libraries, AkkaUtils is not available in the Python API.
4 changes: 2 additions & 2 deletions docs/streaming-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -659,11 +659,11 @@ methods for creating DStreams from files and Akka actors as input sources.
<span class="badge" style="background-color: grey">Python API</span> `fileStream` is not available in the Python API, only `textFileStream` is available.

- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka
actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver
actors by using `AkkaUtils.createStream(ssc, actorProps, actor-name)`. See the [Custom Receiver
Guide](streaming-custom-receivers.html) for more details.

<span class="badge" style="background-color: grey">Python API</span> Since actors are available only in the Java and Scala
libraries, `actorStream` is not available in the Python API.
libraries, `AkkaUtils.createStream` is not available in the Python API.

- **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream.

Expand Down
5 changes: 5 additions & 0 deletions examples/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,11 @@
<artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-akka_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@
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;
import org.apache.spark.streaming.akka.AkkaUtils;
import org.apache.spark.streaming.akka.JavaActorReceiver;

/**
* A sample actor as receiver, is also simplest. This receiver actor
Expand All @@ -56,6 +57,7 @@ public void preStart() {
remotePublisher.tell(new SubscribeReceiver(getSelf()), getSelf());
}

@Override
public void onReceive(Object msg) throws Exception {
store((T) msg);
}
Expand Down Expand Up @@ -100,18 +102,20 @@ public static void main(String[] args) {
String feederActorURI = "akka.tcp://test@" + host + ":" + port + "/user/FeederActor";

/*
* Following is the use of actorStream to plug in custom actor as receiver
* Following is the use of AkkaUtils.createStream 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
* For example: Both AkkaUtils.createStream and JavaSampleActorReceiver are parameterized
* to same type to ensure type safety.
*/
JavaDStream<String> lines = jssc.actorStream(
Props.create(JavaSampleActorReceiver.class, feederActorURI), "SampleReceiver");
JavaDStream<String> lines = AkkaUtils.createStream(
jssc,
Props.create(JavaSampleActorReceiver.class, feederActorURI),
"SampleReceiver");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indent. nvm. my mistake.


// compute wordcount
lines.flatMap(new FlatMapFunction<String, String>() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,12 @@ import scala.collection.mutable.LinkedList
import scala.reflect.ClassTag
import scala.util.Random

import akka.actor.{actorRef2Scala, Actor, ActorRef, Props}
import akka.actor._
import com.typesafe.config.ConfigFactory

import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.SparkConf
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.receiver.ActorReceiver
import org.apache.spark.util.AkkaUtils
import org.apache.spark.streaming.akka.{ActorReceiver, AkkaUtils}

case class SubscribeReceiver(receiverActor: ActorRef)
case class UnsubscribeReceiver(receiverActor: ActorRef)
Expand Down Expand Up @@ -78,8 +78,7 @@ class FeederActor extends Actor {
*
* @see [[org.apache.spark.examples.streaming.FeederActor]]
*/
class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
extends ActorReceiver {
class SampleActorReceiver[T](urlOfPublisher: String) extends ActorReceiver {

lazy private val remotePublisher = context.actorSelection(urlOfPublisher)

Expand Down Expand Up @@ -108,9 +107,13 @@ object FeederActor {
}
val Seq(host, port) = args.toSeq

val conf = new SparkConf
val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = conf,
securityManager = new SecurityManager(conf))._1
val akkaConf = ConfigFactory.parseString(
s"""akka.actor.provider = "akka.remote.RemoteActorRefProvider"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe this change will resolve SPARK-12786.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zsxwing Could you also tag that JIRA in this PR and resolve it when this is closed.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated the title

|akka.remote.enabled-transports = ["akka.remote.netty.tcp"]
|akka.remote.netty.tcp.hostname = "$host"
|akka.remote.netty.tcp.port = $port
|""".stripMargin)
val actorSystem = ActorSystem("test", akkaConf)
val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor")

println("Feeder started as:" + feeder)
Expand All @@ -121,6 +124,7 @@ object FeederActor {

/**
* A sample word count program demonstrating the use of plugging in
*
* Actor as Receiver
* Usage: ActorWordCount <hostname> <port>
* <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
Expand All @@ -146,20 +150,21 @@ object ActorWordCount {
val ssc = new StreamingContext(sparkConf, Seconds(2))

/*
* Following is the use of actorStream to plug in custom actor as receiver
* Following is the use of AkkaUtils.createStream 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
* to ensure the type safety, i.e type of data received and InputDStream
* should be same.
*
* For example: Both actorStream and SampleActorReceiver are parameterized
* For example: Both AkkaUtils.createStream and SampleActorReceiver are parameterized
* to same type to ensure type safety.
*/

val lines = ssc.actorStream[String](
Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format(
host, port.toInt))), "SampleReceiver")
val lines = AkkaUtils.createStream[String](
ssc,
Props(classOf[SampleActorReceiver[String]],
"akka.tcp://test@%s:%s/user/FeederActor".format(host, port.toInt)),
"SampleReceiver")

// compute wordcount
lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,9 @@ import akka.actor.actorRef2Scala
import akka.util.ByteString
import akka.zeromq._
import akka.zeromq.Subscribe
import com.typesafe.config.ConfigFactory

import org.apache.spark.SparkConf
import org.apache.spark.{SparkConf, TaskContext}
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.zeromq._

Expand Down Expand Up @@ -69,10 +70,10 @@ object SimpleZeroMQPublisher {
*
* To run this example locally, you may run publisher as
* `$ bin/run-example \
* org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
* org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.0.1:1234 foo`
* and run the example as
* `$ bin/run-example \
* org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo`
* org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.0.1:1234 foo`
*/
// scalastyle:on
object ZeroMQWordCount {
Expand All @@ -90,7 +91,11 @@ object ZeroMQWordCount {
def bytesToStringIterator(x: Seq[ByteString]): Iterator[String] = x.map(_.utf8String).iterator

// For this stream, a zeroMQ publisher should be running.
val lines = ZeroMQUtils.createStream(ssc, 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()
Expand Down
73 changes: 73 additions & 0 deletions external/akka/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
<?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_2.10</artifactId>
<version>2.0.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-akka_2.10</artifactId>
<properties>
<sbt.project.name>streaming-akka</sbt.project.name>
</properties>
<packaging>jar</packaging>
<name>Spark Project External Akka</name>
<url>http://spark.apache.org/</url>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
<artifactId>akka-actor_${scala.binary.version}</artifactId>
<version>${akka.version}</version>
</dependency>
<dependency>
<groupId>${akka.group}</groupId>
<artifactId>akka-remote_${scala.binary.version}</artifactId>
<version>${akka.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
</build>
</project>
Loading