aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
authorDavies Liu <davies@databricks.com>2015-02-02 19:16:27 -0800
committerTathagata Das <tathagata.das1565@gmail.com>2015-02-02 19:16:27 -0800
commit0561c4544967fb853419f32e014fac9b8879b0db (patch)
tree54f64d9481de296bcb7676f82306896007b489b2 /python
parent554403fd913685da879cf6a280c58a9fad19448a (diff)
downloadspark-0561c4544967fb853419f32e014fac9b8879b0db.tar.gz
spark-0561c4544967fb853419f32e014fac9b8879b0db.tar.bz2
spark-0561c4544967fb853419f32e014fac9b8879b0db.zip
[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=<function utf8_decoder>, valueDecoder=<function utf8_decoder>) | 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 <davies@databricks.com> Author: Tathagata Das <tdas@databricks.com> 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
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/serializers.py9
-rw-r--r--python/pyspark/streaming/kafka.py83
-rw-r--r--python/pyspark/tests.py10
3 files changed, 100 insertions, 2 deletions
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index b8bda83517..0ffb41d02f 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -70,6 +70,7 @@ class SpecialLengths(object):
PYTHON_EXCEPTION_THROWN = -2
TIMING_DATA = -3
END_OF_STREAM = -4
+ NULL = -5
class Serializer(object):
@@ -133,6 +134,8 @@ class FramedSerializer(Serializer):
def _write_with_length(self, obj, stream):
serialized = self.dumps(obj)
+ if serialized is None:
+ raise ValueError("serialized value should not be None")
if len(serialized) > (1 << 31):
raise ValueError("can not serialize object larger than 2G")
write_int(len(serialized), stream)
@@ -145,8 +148,10 @@ class FramedSerializer(Serializer):
length = read_int(stream)
if length == SpecialLengths.END_OF_DATA_SECTION:
raise EOFError
+ elif length == SpecialLengths.NULL:
+ return None
obj = stream.read(length)
- if obj == "":
+ if len(obj) < length:
raise EOFError
return self.loads(obj)
@@ -484,6 +489,8 @@ class UTF8Deserializer(Serializer):
length = read_int(stream)
if length == SpecialLengths.END_OF_DATA_SECTION:
raise EOFError
+ elif length == SpecialLengths.NULL:
+ return None
s = stream.read(length)
return s.decode("utf-8") if self.use_unicode else s
diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py
new file mode 100644
index 0000000000..19ad71f99d
--- /dev/null
+++ b/python/pyspark/streaming/kafka.py
@@ -0,0 +1,83 @@
+#
+# 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.
+#
+
+from py4j.java_collections import MapConverter
+from py4j.java_gateway import java_import, Py4JError
+
+from pyspark.storagelevel import StorageLevel
+from pyspark.serializers import PairDeserializer, NoOpSerializer
+from pyspark.streaming import DStream
+
+__all__ = ['KafkaUtils', 'utf8_decoder']
+
+
+def utf8_decoder(s):
+ """ Decode the unicode as UTF-8 """
+ return s and s.decode('utf-8')
+
+
+class KafkaUtils(object):
+
+ @staticmethod
+ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={},
+ storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2,
+ keyDecoder=utf8_decoder, valueDecoder=utf8_decoder):
+ """
+ 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 kafkaParams: Additional params for Kafka
+ :param storageLevel: RDD storage level.
+ :param keyDecoder: A function used to decode key (default is utf8_decoder)
+ :param valueDecoder: A function used to decode value (default is utf8_decoder)
+ :return: A DStream object
+ """
+ java_import(ssc._jvm, "org.apache.spark.streaming.kafka.KafkaUtils")
+
+ kafkaParams.update({
+ "zookeeper.connect": zkQuorum,
+ "group.id": groupId,
+ "zookeeper.connection.timeout.ms": "10000",
+ })
+ if not isinstance(topics, dict):
+ raise TypeError("topics should be dict")
+ jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client)
+ jparam = MapConverter().convert(kafkaParams, ssc.sparkContext._gateway._gateway_client)
+ jlevel = ssc._sc._getJavaStorageLevel(storageLevel)
+
+ def getClassByName(name):
+ return ssc._jvm.org.apache.spark.util.Utils.classForName(name)
+
+ try:
+ array = getClassByName("[B")
+ decoder = getClassByName("kafka.serializer.DefaultDecoder")
+ jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, array, decoder, decoder,
+ jparam, jtopics, jlevel)
+ except Py4JError, e:
+ # TODO: use --jar once it also work on driver
+ if not e.message or 'call a package' in e.message:
+ print "No kafka package, please put the assembly jar into classpath:"
+ print " $ bin/spark-submit --driver-class-path external/kafka-assembly/target/" + \
+ "scala-*/spark-streaming-kafka-assembly-*.jar"
+ raise e
+ ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
+ stream = DStream(jstream, ssc, ser)
+ return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v)))
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index fef6c92875..c7d0622d65 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -47,9 +47,10 @@ else:
from pyspark.conf import SparkConf
from pyspark.context import SparkContext
+from pyspark.rdd import RDD
from pyspark.files import SparkFiles
from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \
- CloudPickleSerializer, CompressedSerializer
+ CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer
from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter
from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \
UserDefinedType, DoubleType
@@ -716,6 +717,13 @@ class RDDTests(ReusedPySparkTestCase):
wr_s21 = rdd.sample(True, 0.4, 21).collect()
self.assertNotEqual(set(wr_s11), set(wr_s21))
+ def test_null_in_rdd(self):
+ jrdd = self.sc._jvm.PythonUtils.generateRDDWithNull(self.sc._jsc)
+ rdd = RDD(jrdd, self.sc, UTF8Deserializer())
+ self.assertEqual([u"a", None, u"b"], rdd.collect())
+ rdd = RDD(jrdd, self.sc, NoOpSerializer())
+ self.assertEqual(["a", None, "b"], rdd.collect())
+
def test_multiple_python_java_RDD_conversions(self):
# Regression test for SPARK-5361
data = [