From 0561c4544967fb853419f32e014fac9b8879b0db Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 2 Feb 2015 19:16:27 -0800 Subject: [SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python This PR brings the Python API for Spark Streaming Kafka data source. ``` class KafkaUtils(__builtin__.object) | Static methods defined here: | | createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False, 2), keyDecoder=, valueDecoder=) | Create an input stream that pulls messages from a Kafka Broker. | | :param ssc: StreamingContext object | :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). | :param groupId: The group id for this consumer. | :param topics: Dict of (topic_name -> numPartitions) to consume. | Each partition is consumed in its own thread. | :param storageLevel: RDD storage level. | :param keyDecoder: A function used to decode key | :param valueDecoder: A function used to decode value | :return: A DStream object ``` run the example: ``` bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test ``` Author: Davies Liu Author: Tathagata Das Closes #3715 from davies/kafka and squashes the following commits: d93bfe0 [Davies Liu] Update make-distribution.sh 4280d04 [Davies Liu] address comments e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka f257071 [Davies Liu] add tests for null in RDD 23b039a [Davies Liu] address comments 9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka a74da87 [Davies Liu] address comments dc1eed0 [Davies Liu] Update kafka_wordcount.py 31e2317 [Davies Liu] Update kafka_wordcount.py 370ba61 [Davies Liu] Update kafka.py 97386b3 [Davies Liu] address comment 2c567a5 [Davies Liu] update logging and comment 33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api aea8953 [Tathagata Das] Kafka-assembly for Python API eea16a7 [Davies Liu] refactor f6ce899 [Davies Liu] add example and fix bugs 98c8d17 [Davies Liu] fix python style 5697a01 [Davies Liu] bypass decoder in scala 048dbe6 [Davies Liu] fix python style 75d485e [Davies Liu] add mqtt 07923c4 [Davies Liu] support kafka in Python --- .../src/main/python/streaming/kafka_wordcount.py | 54 ++++++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 examples/src/main/python/streaming/kafka_wordcount.py (limited to 'examples/src/main/python') diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py new file mode 100644 index 0000000000..ed398a82b8 --- /dev/null +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -0,0 +1,54 @@ +# +# 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: network_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first, see + http://kafka.apache.org/documentation.html#quickstart + + and then run the example + `$ bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/\ + spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ + localhost:2181 test` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: kafka_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingKafkaWordCount") + ssc = StreamingContext(sc, 1) + + zkQuorum, topic = sys.argv[1:] + kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) + 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() -- cgit v1.2.3