-
Notifications
You must be signed in to change notification settings - Fork 28.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-8378][Streaming]Add the Python API for Flume #6830
Changes from 1 commit
9f33873
0336579
4762c34
b8d5551
14ba0ff
152364c
01cbb3d
ce85e83
b96b0de
7a55837
f9ee681
9d5821e
e93736b
0449723
f1bf3c0
78dfdac
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
# | ||
# 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. | ||
# | ||
|
||
""" | ||
Counts words in UTF8 encoded, '\n' delimited text received from the network every second. | ||
Usage: flume_wordcount.py <hostname> <port> | ||
|
||
To run this on your local machine, you need to setup Flume first, see | ||
https://flume.apache.org/documentation.html | ||
|
||
and then run the example | ||
`$ bin/spark-submit --jars external/flume-assembly/target/scala-*/\ | ||
spark-streaming-flume-assembly-*.jar examples/src/main/python/streaming/flume_wordcount.py \ | ||
localhost 12345 | ||
""" | ||
from __future__ import print_function | ||
|
||
import sys | ||
|
||
from pyspark import SparkContext | ||
from pyspark.streaming import StreamingContext | ||
from pyspark.streaming.flume import FlumeUtils | ||
|
||
if __name__ == "__main__": | ||
if len(sys.argv) != 3: | ||
print("Usage: flume_wordcount.py <hostname> <port>", file=sys.stderr) | ||
exit(-1) | ||
|
||
sc = SparkContext(appName="PythonStreamingFlumeWordCount") | ||
ssc = StreamingContext(sc, 1) | ||
|
||
hostname, port = sys.argv[1:] | ||
kvs = FlumeUtils.createStream(ssc, hostname, int(port)) | ||
lines = kvs.map(lambda x: x[1]) | ||
counts = lines.flatMap(lambda line: line.split(" ")) \ | ||
.map(lambda word: (word, 1)) \ | ||
.reduceByKey(lambda a, b: a+b) | ||
counts.pprint() | ||
|
||
ssc.start() | ||
ssc.awaitTermination() |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,134 @@ | ||
<?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>1.5.0-SNAPSHOT</version> | ||
<relativePath>../../pom.xml</relativePath> | ||
</parent> | ||
|
||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-streaming-flume-assembly_2.10</artifactId> | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi guys, Sorry I'm late to the party, but why is this new assembly necessary? It creates an 80MB jar file that repackages a bunch of things already present in the Spark assembly (e.g. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nope, none of the fume stuff is present in the spark-assembly. That is On Mon, Jul 6, 2015 at 2:02 PM, Marcelo Vanzin notifications@github.com
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Well, two things:
So any way you look at it, there is still something to be fixed here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
On Mon, Jul 6, 2015 at 2:19 PM, Marcelo Vanzin notifications@github.com
|
||
<packaging>jar</packaging> | ||
<name>Spark Project External Flume Assembly</name> | ||
<url>http://spark.apache.org/</url> | ||
|
||
<properties> | ||
<sbt.project.name>streaming-flume-assembly</sbt.project.name> | ||
</properties> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-streaming-flume_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
</dependency> | ||
<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.avro</groupId> | ||
<artifactId>avro</artifactId> | ||
<version>${avro.version}</version> | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The dependencies of
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems like an avro bug. Can you file a jira for Avro? Avro should be compatible within the same minor version - 1.7.x There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do you know where the other version of avro is coming up ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I think
Actually "mvn dependency:tree" shows both |
||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.avro</groupId> | ||
<artifactId>avro-ipc</artifactId> | ||
<version>${avro.version}</version> | ||
<exclusions> | ||
<exclusion> | ||
<groupId>io.netty</groupId> | ||
<artifactId>netty</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>org.mortbay.jetty</groupId> | ||
<artifactId>jetty</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>org.mortbay.jetty</groupId> | ||
<artifactId>jetty-util</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>org.mortbay.jetty</groupId> | ||
<artifactId>servlet-api</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>org.apache.velocity</groupId> | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do you need to pull all these in? Doesn't the spark-streaming-flume artifact pull these in anyway? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These are exclusions.. not pulling in. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ah, guess I need more coffee |
||
<artifactId>velocity</artifactId> | ||
</exclusion> | ||
</exclusions> | ||
</dependency> | ||
</dependencies> | ||
|
||
<build> | ||
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory> | ||
<plugins> | ||
<plugin> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-shade-plugin</artifactId> | ||
<configuration> | ||
<shadedArtifactAttached>false</shadedArtifactAttached> | ||
<artifactSet> | ||
<includes> | ||
<include>*:*</include> | ||
</includes> | ||
</artifactSet> | ||
<filters> | ||
<filter> | ||
<artifact>*:*</artifact> | ||
<excludes> | ||
<exclude>META-INF/*.SF</exclude> | ||
<exclude>META-INF/*.DSA</exclude> | ||
<exclude>META-INF/*.RSA</exclude> | ||
</excludes> | ||
</filter> | ||
</filters> | ||
</configuration> | ||
<executions> | ||
<execution> | ||
<phase>package</phase> | ||
<goals> | ||
<goal>shade</goal> | ||
</goals> | ||
<configuration> | ||
<transformers> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer"> | ||
<resource>reference.conf</resource> | ||
</transformer> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer"> | ||
<resource>log4j.properties</resource> | ||
</transformer> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/> | ||
</transformers> | ||
</configuration> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,10 +18,16 @@ | |
package org.apache.spark.streaming.flume | ||
|
||
import java.net.InetSocketAddress | ||
import java.io.{DataOutputStream, ByteArrayOutputStream} | ||
import java.util.{List => JList, Map => JMap} | ||
|
||
import scala.collection.JavaConversions._ | ||
|
||
import org.apache.spark.api.java.function.PairFunction | ||
import org.apache.spark.api.python.PythonRDD | ||
import org.apache.spark.storage.StorageLevel | ||
import org.apache.spark.streaming.StreamingContext | ||
import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} | ||
import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaReceiverInputDStream, JavaStreamingContext} | ||
import org.apache.spark.streaming.dstream.ReceiverInputDStream | ||
|
||
|
||
|
@@ -236,3 +242,71 @@ object FlumeUtils { | |
createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) | ||
} | ||
} | ||
|
||
/** | ||
* This is a helper class that wraps the methods in FlumeUtils into more Python-friendly class and | ||
* function so that it can be easily instantiated and called from Python's FlumeUtils. | ||
*/ | ||
private class FlumeUtilsPythonHelper { | ||
|
||
def createStream( | ||
jssc: JavaStreamingContext, | ||
hostname: String, | ||
port: Int, | ||
storageLevel: StorageLevel, | ||
enableDecompression: Boolean | ||
): JavaPairDStream[Array[Byte], Array[Byte]] = { | ||
val dstream = FlumeUtils.createStream(jssc, hostname, port, storageLevel, enableDecompression) | ||
FlumeUtilsPythonHelper.toDStreamForPython(dstream) | ||
} | ||
|
||
def createPollingStream( | ||
jssc: JavaStreamingContext, | ||
hosts: JList[String], | ||
ports: JList[Int], | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not very intuitive. Why not take a list of (host, port) tuples? Is there an issue passing tuples from Python? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Java-friendliness. So that there are no issues that can arise when calling from python through py4j. This is not meant to be called by public so this is sort-of-okay. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. hmm, ok. I generally don't like APIs that have two lists that have index based matching. Perhaps we can pass in a Since is not really user-facing, I am sort-of-ok with this, but I'd prefer to have a cleaner API. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
It cannot be a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
storageLevel: StorageLevel, | ||
maxBatchSize: Int, | ||
parallelism: Int | ||
): JavaPairDStream[Array[Byte], Array[Byte]] = { | ||
assert(hosts.length == ports.length) | ||
val addresses = hosts.zip(ports).map { | ||
case (host, port) => new InetSocketAddress(host, port) | ||
} | ||
val dstream = FlumeUtils.createPollingStream( | ||
jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) | ||
FlumeUtilsPythonHelper.toDStreamForPython(dstream) | ||
} | ||
|
||
} | ||
|
||
private object FlumeUtilsPythonHelper { | ||
|
||
private def stringMapToByteArray(map: JMap[CharSequence, CharSequence]): Array[Byte] = { | ||
val byteStream = new ByteArrayOutputStream() | ||
val output = new DataOutputStream(byteStream) | ||
try { | ||
output.writeInt(map.size) | ||
map.foreach { kv => | ||
PythonRDD.writeUTF(kv._1.toString, output) | ||
PythonRDD.writeUTF(kv._2.toString, output) | ||
} | ||
byteStream.toByteArray | ||
} | ||
finally { | ||
output.close() | ||
} | ||
} | ||
|
||
private def toDStreamForPython(dstream: JavaReceiverInputDStream[SparkFlumeEvent]): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The name isnt very intuitive, could be |
||
JavaPairDStream[Array[Byte], Array[Byte]] = { | ||
dstream.mapToPair(new PairFunction[SparkFlumeEvent, Array[Byte], Array[Byte]] { | ||
override def call(sparkEvent: SparkFlumeEvent): (Array[Byte], Array[Byte]) = { | ||
val event = sparkEvent.event | ||
val byteBuffer = event.getBody | ||
val body = new Array[Byte](byteBuffer.remaining()) | ||
byteBuffer.get(body) | ||
(stringMapToByteArray(event.getHeaders), body) | ||
} | ||
}) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is the decoding logic the same here too? UTF-8 encoded string, or custom decoding function? If yes, we should move that snippet explaining this outside of both approaches and specify that it is applicable to both.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I updated the doc. Because there is no a Python example for
FlumeUtils.createPollingStream
(it requires the user installing the Spark sink jar to Flume, so we cannot write an out-of-the-box example), I just copied the description of the encoding function to here.