Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/2538#discussion_r18378148
--- Diff: python/pyspark/streaming/dstream.py ---
@@ -0,0 +1,624 @@
+#
+# 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 itertools import chain, ifilter, imap
+import operator
+import time
+from datetime import datetime
+
+from py4j.protocol import Py4JJavaError
+
+from pyspark import RDD
+from pyspark.storagelevel import StorageLevel
+from pyspark.streaming.util import rddToFileName, TransformFunction
+from pyspark.rdd import portable_hash
+from pyspark.resultiterable import ResultIterable
+
+__all__ = ["DStream"]
+
+
+class DStream(object):
+ """
+ A Discretized Stream (DStream), the basic abstraction in Spark
Streaming,
+ is a continuous sequence of RDDs (of the same type) representing a
+ continuous stream of data (see L{RDD} in the Spark core documentation
+ for more details on RDDs).
+
+ DStreams can either be created from live data (such as, data from TCP
+ sockets, Kafka, Flume, etc.) using a L{StreamingContext} or it can be
+ generated by transforming existing DStreams using operations such as
+ `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming
+ program is running, each DStream periodically generates a RDD, either
+ from live data or by transforming the RDD generated by a parent
DStream.
+
+ DStreams internally is characterized by a few basic properties:
+ - A list of other DStreams that the DStream depends on
+ - A time interval at which the DStream generates an RDD
+ - A function that is used to generate an RDD after each time interval
+ """
+ def __init__(self, jdstream, ssc, jrdd_deserializer):
+ self._jdstream = jdstream
+ self._ssc = ssc
+ self.ctx = ssc._sc
+ self._jrdd_deserializer = jrdd_deserializer
+ self.is_cached = False
+ self.is_checkpointed = False
+
+ def context(self):
+ """
+ Return the StreamingContext associated with this DStream
+ """
+ return self._ssc
+
+ def count(self):
+ """
+ Return a new DStream in which each RDD has a single element
+ generated by counting each RDD of this DStream.
+ """
+ return self.mapPartitions(lambda i: [sum(1 for _ in i)])._sum()
+
+ def _sum(self):
+ """
+ Add up the elements in this DStream.
+ """
+ return self.mapPartitions(lambda x: [sum(x)]).reduce(operator.add)
+
+ def filter(self, f):
+ """
+ Return a new DStream containing only the elements that satisfy
predicate.
+ """
+ def func(iterator):
+ return ifilter(f, iterator)
+ return self.mapPartitions(func, True)
+
+ def flatMap(self, f, preservesPartitioning=False):
+ """
+ Return a new DStream by applying a function to all elements of
+ this DStream, and then flattening the results
+ """
+ def func(s, iterator):
+ return chain.from_iterable(imap(f, iterator))
+ return self.mapPartitionsWithIndex(func, preservesPartitioning)
+
+ def map(self, f, preservesPartitioning=False):
+ """
+ Return a new DStream by applying a function to each element of
DStream.
+ """
+ def func(iterator):
+ return imap(f, iterator)
+ return self.mapPartitions(func, preservesPartitioning)
+
+ def mapPartitions(self, f, preservesPartitioning=False):
+ """
+ Return a new DStream in which each RDD is generated by applying
+ mapPartitions() to each RDDs of this DStream.
+ """
+ def func(s, iterator):
+ return f(iterator)
+ return self.mapPartitionsWithIndex(func, preservesPartitioning)
+
+ def mapPartitionsWithIndex(self, f, preservesPartitioning=False):
+ """
+ Return a new DStream in which each RDD is generated by applying
+ mapPartitionsWithIndex() to each RDDs of this DStream.
+ """
+ return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f,
preservesPartitioning))
+
+ def reduce(self, func):
+ """
+ Return a new DStream in which each RDD has a single element
+ generated by reducing each RDD of this DStream.
+ """
+ return self.map(lambda x: (None, x)).reduceByKey(func,
1).map(lambda x: x[1])
+
+ def reduceByKey(self, func, numPartitions=None):
+ """
+ Return a new DStream by applying reduceByKey to each RDD.
+ """
+ if numPartitions is None:
+ numPartitions = self.ctx.defaultParallelism
+ return self.combineByKey(lambda x: x, func, func, numPartitions)
+
+ def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
+ numPartitions=None):
+ """
+ Return a new DStream by applying combineByKey to each RDD.
+ """
+ if numPartitions is None:
+ numPartitions = self.ctx.defaultParallelism
+
+ def func(rdd):
+ return rdd.combineByKey(createCombiner, mergeValue,
mergeCombiners, numPartitions)
+ return self.transform(func)
+
+ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
+ """
+ Return a copy of the DStream in which each RDD are partitioned
+ using the specified partitioner.
+ """
+ return self.transform(lambda rdd: rdd.partitionBy(numPartitions,
partitionFunc))
+
+ def foreachRDD(self, func):
+ """
+ Apply a function to each RDD in this DStream.
+ """
+ jfunc = TransformFunction(self.ctx, func, self._jrdd_deserializer)
+ api = self._ssc._jvm.PythonDStream
+ api.callForeachRDD(self._jdstream, jfunc)
+
+ def pprint(self):
+ """
+ Print the first ten elements of each RDD generated in this DStream.
+ """
+ def takeAndPrint(time, rdd):
+ taken = rdd.take(11)
+ print "-------------------------------------------"
+ print "Time: %s" % time
+ print "-------------------------------------------"
+ for record in taken[:10]:
+ print record
+ if len(taken) > 10:
+ print "..."
+ print
+
+ self.foreachRDD(takeAndPrint)
+
+ def mapValues(self, f):
+ """
+ Return a new DStream by applying a map function to the value of
+ each key-value pairs in 'this' DStream without changing the key.
+ """
+ map_values_fn = lambda (k, v): (k, f(v))
+ return self.map(map_values_fn, preservesPartitioning=True)
+
+ def flatMapValues(self, f):
+ """
+ Return a new DStream by applying a flatmap function to the value
+ of each key-value pairs in 'this' DStream without changing the key.
+ """
+ flat_map_fn = lambda (k, v): ((k, x) for x in f(v))
+ return self.flatMap(flat_map_fn, preservesPartitioning=True)
+
+ def glom(self):
+ """
+ Return a new DStream in which RDD is generated by applying glom()
+ to RDD of this DStream.
+ """
+ def func(iterator):
+ yield list(iterator)
+ return self.mapPartitions(func)
+
+ def cache(self):
+ """
+ Persist the RDDs of this DStream with the default storage level
+ (C{MEMORY_ONLY_SER}).
+ """
+ self.is_cached = True
+ self.persist(StorageLevel.MEMORY_ONLY_SER)
+ return self
+
+ def persist(self, storageLevel):
+ """
+ Persist the RDDs of this DStream with the given storage level
+ """
+ self.is_cached = True
+ javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
+ self._jdstream.persist(javaStorageLevel)
+ return self
+
+ def checkpoint(self, interval):
+ """
+ Enable periodic checkpointing of RDDs of this DStream
+
+ @param interval: time in seconds, after each period of that,
generated
+ RDD will be checkpointed
+ """
+ self.is_checkpointed = True
+ self._jdstream.checkpoint(self._ssc._jduration(interval))
+ return self
+
+ def groupByKey(self, numPartitions=None):
+ """
+ Return a new DStream by applying groupByKey on each RDD.
+ """
+ if numPartitions is None:
+ numPartitions = self.ctx.defaultParallelism
+ return self.transform(lambda rdd: rdd.groupByKey(numPartitions))
+
+ def countByValue(self):
+ """
+ Return a new DStream in which each RDD contains the counts of each
+ distinct value in each RDD of this DStream.
+ """
+ return self.map(lambda x: (x, None)).reduceByKey(lambda x, y:
None).count()
+
+ def saveAsTextFiles(self, prefix, suffix=None):
+ """
+ Save each RDD in this DStream as at text file, using string
+ representation of elements.
+ """
+ def saveAsTextFile(t, rdd):
+ path = rddToFileName(prefix, suffix, t)
+ try:
+ rdd.saveAsTextFile(path)
+ except Py4JJavaError as e:
+ # after recovered from checkpointing, the foreachRDD may
+ # be called twice
+ if 'FileAlreadyExistsException' not in str(e):
+ raise
+ return self.foreachRDD(saveAsTextFile)
+
+ def _saveAsPickleFiles(self, prefix, suffix=None):
--- End diff --
This is mirror the RDD api of saveAsPickFile. I am not entirely sure
whether to expose this, but not point removing the implementation, so just
having it as private should be fine for now.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]