You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by davies <gi...@git.apache.org> on 2014/09/26 01:52:16 UTC

[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

GitHub user davies opened a pull request:

    https://github.com/apache/spark/pull/2538

    [WIP] [SPARK-2377] Python API for Streaming

    This patch bring Python API for Streaming, WIP.
    
    TODO:
    updateStateByKey()
    windowXXX()
    
    This patch is based on work from @giwa

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/davies/spark streaming

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2538.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2538
    
----
commit e8c7bfc556da45d33f9ffecf8c6b802fe7a7e49c
Author: giwa <ug...@gmail.com>
Date:   2014-08-11T11:31:59Z

    remove export PYSPARK_PYTHON in spark submit

commit bdde697368cee7c06fcbcf4f2102fedf3a58536f
Author: giwa <ug...@gmail.com>
Date:   2014-08-11T11:42:08Z

    removed unnesessary changes

commit a65f3021fc8aa5f82889a18a728eed3c901996d0
Author: giwa <ug...@gmail.com>
Date:   2014-08-11T12:32:28Z

    edited the comment to add more precise description

commit 90a6484066ec2c157db6650d470e0b66cf42b342
Author: giwa <ug...@gmail.com>
Date:   2014-08-11T23:34:12Z

    added mapValues and flatMapVaules WIP for glom and mapPartitions test

commit 0704b86a9963c1d62b1934ce2fb47094b3fb03d3
Author: giwa <ug...@gmail.com>
Date:   2014-08-14T04:04:26Z

    WIP: solved partitioned and None is not recognized

commit 080541a6d77cb85f788c297670cca24fbbc9f9b5
Author: giwa <ug...@gmail.com>
Date:   2014-08-14T09:19:46Z

    broke something

commit 2112638167e258609551df6e6036f33e08ff82e3
Author: giwa <ug...@gmail.com>
Date:   2014-08-15T01:07:10Z

    all tests are passed if numSlice is 2 and the numver of each input is over 4

commit 536def42b9c8b0b81499e5e06d22b813f18d0bdd
Author: giwa <ug...@gmail.com>
Date:   2014-08-15T06:42:34Z

    basic function test cases are passed

commit a14c7e1a59370949a5f1eab16e448cc0012fa65e
Author: giwa <ug...@gmail.com>
Date:   2014-08-15T06:46:45Z

    modified streaming test case to add coment

commit e3033fcdd24258eb3836c0c07e5c959c3dfde7d2
Author: giwa <ug...@gmail.com>
Date:   2014-08-15T18:28:39Z

    remove waste duplicated code

commit 89ae38a0d6bc299ebb9aa81c7510812874ce7879
Author: giwa <ug...@gmail.com>
Date:   2014-08-16T00:10:56Z

    added saveAsTextFiles and saveAsPickledFiles

commit ea9c8731b3d997ead7015d721c66231064e19ff9
Author: giwa <ug...@gmail.com>
Date:   2014-08-16T05:30:58Z

    added TODO coments

commit d8b593b20351d32d4ac3948778bf2ebbab86879f
Author: giwa <ug...@gmail.com>
Date:   2014-08-18T07:30:17Z

    add comments

commit e7ebb08da3c59102cfad08ce4d687e56d02a0edf
Author: giwa <ug...@gmail.com>
Date:   2014-08-18T07:35:50Z

    removed wasted print in DStream

commit 636090ac5323cdde6c72d48336b716693a80e010
Author: giwa <ug...@gmail.com>
Date:   2014-08-18T20:24:17Z

    added sparkContext as input parameter in StreamingContext

commit a3d2379d79fdb8573963564f5c5be98558e495f2
Author: giwa <ug...@gmail.com>
Date:   2014-08-18T21:39:45Z

    added gorupByKey testcase

commit 665bfdb48523ecb7aa5174341a74c55c2088a891
Author: giwa <ug...@gmail.com>
Date:   2014-08-18T22:12:31Z

    added testcase for combineByKey

commit 5c3a683efb76c49e6441672272bc029ecfbb687a
Author: Ken <ug...@gmail.com>
Date:   2014-07-09T01:31:41Z

    initial commit for pySparkStreaming

commit e497b9bfe6ba96db46122aa369b5dba528524c2e
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-15T22:41:52Z

    comment PythonDStream.PairwiseDStream

commit 6e0d9c749e7ef0067a6cd7ae9d21e8b599e32d54
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T00:19:20Z

    modify dstream.py to fix indent error

commit 9af03f40bbb9d04cfe66398a8632e4398214e3d7
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T04:08:43Z

    added reducedByKey not working yet

commit dcf243f1cd0e7e5e47fb5b4ef9f269a344291f1b
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:07:42Z

    implementing transform function in Python

commit c5518b42c6f5b3832a508eb302c34f84cf15b864
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:12:53Z

    modified the code base on comment in https://github.com/tdas/spark/pull/10

commit 375817561de68b54be4b41ddbf6dbfc352d59360
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:17:02Z

    add coment for hack why PYSPARK_PYTHON is needed in spark-submit

commit e551e1355132ed239baf4edd51f3e275222362cc
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:19:13Z

    add coment for hack why PYSPARK_PYTHON is needed in spark-submit

commit 2adca8419495eaaafab2677c8e2ba6f9588dfeb0
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:24:08Z

    remove not implemented DStream functions in python

commit 5594bd43622adeac153642d600ab5585c5f7a2bb
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T18:35:59Z

    revert pom.xml

commit 490e338374bef5265796332f7b0a5defe6839754
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T19:15:06Z

    sorted the import following Spark coding convention

commit 856d98e67b7df23f9c86da6c42795238c8dbcdc4
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T19:19:42Z

    add empty line

commit 4ce4058a216de9118772df8b46085665bf28a51c
Author: Ken Takagiwa <ke...@kens-macbook-pro.local>
Date:   2014-07-16T22:40:42Z

    remove unused import in python

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57045280
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20912/consoleFull) for   PR 2538 at commit [`b32774c`](https://github.com/apache/spark/commit/b32774cc3cc7493b360bd9e5b8b01df28968d0c2).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18201664
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -413,7 +413,7 @@ class StreamingContext private[streaming] (
           dstreams: Seq[DStream[_]],
           transformFunc: (Seq[RDD[_]], Time) => RDD[T]
         ): DStream[T] = {
    -    new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc))
    +    new TransformedDStream[T](dstreams, transformFunc)
    --- End diff --
    
    Without this change, ssc.transform() will failed to serialize transformFunc, because the callback function from python is not serializable.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18132355
  
    --- Diff: examples/src/main/python/streaming/wordcount.py ---
    @@ -0,0 +1,21 @@
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 2:
    +        print >> sys.stderr, "Usage: wordcount <directory>"
    +        exit(-1)
    +
    +    sc = SparkContext(appName="PythonStreamingWordCount")
    +    ssc = StreamingContext(sc, 1)
    +
    +    lines = ssc.textFileStream(sys.argv[1])
    +    counts = lines.flatMap(lambda line: line.split(" "))\
    +                  .map(lambda x: (x, 1))\
    +                  .reduceByKey(lambda a, b: a+b)
    +    counts.pyprint()
    --- End diff --
    
    counts.pyprint() should be counts.pprint()
    
    ```
        def pprint(self):
            """
            Print the first ten elements of each RDD generated in this DStream. This is an output
            operator, so this DStream will be registered as an output stream and there materialized.
            """
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194030
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    +        """
    +        Return the first RDD in the stream.
    +        """
    +        return self._take(1)[0]
    +
    +    def _take(self, n):
    +        """
    +        Return the first `n` RDDs in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        self.foreachRDD(take)
    +
    +        self._ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self._ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            result.append(r)
    +        self.foreachRDD(get_output)
    +        return result
    +
    +    def mapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a map function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        map_values_fn = lambda (k, v): (k, f(v))
    +        return self.map(map_values_fn, preservesPartitioning=True)
    +
    +    def flatMapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a flatMap function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        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. Applying glom() to an RDD coalesces all
    +        elements within each partition into an list.
    +        """
    +        def func(iterator):
    +            yield list(iterator)
    +        return self.mapPartitions(func)
    +
    +    def cache(self):
    +        """
    +        Persist 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):
    +        """
    +        Set this DStream's storage level to persist its values across operations
    +        after the first time it is computed. This can only be used to assign
    +        a new storage level if the DStream does not have a storage level set yet.
    +        """
    +        self.is_cached = True
    +        javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
    +        self._jdstream.persist(javaStorageLevel)
    +        return self
    +
    +    def checkpoint(self, interval):
    +        """
    +        Mark this DStream for checkpointing. It will be saved to a file inside the
    +        checkpoint directory set with L{SparkContext.setCheckpointDir()}
    +
    +        @param interval: time in seconds, after which 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 which contains group the values for each key in the
    +        DStream into a single sequence.
    +        Hash-partitions the resulting RDD with into numPartitions partitions in
    +        the DStream.
    +
    +        Note: If you are grouping in order to perform an aggregation (such as a
    +        sum or average) over each key, using reduceByKey will provide much
    +        better performance.
    +        """
    +        return self.transform(lambda rdd: rdd.groupByKey(numPartitions))
    +
    +    def countByValue(self):
    +        """
    +        Return new DStream which contains the count of each unique value in this
    +        DStreeam as a (value, count) pairs.
    +        """
    +        return self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None).count()
    +
    +    def saveAsTextFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a text file, using string representations of elements.
    +        """
    +
    +        def saveAsTextFile(time, rdd):
    +            """
    +            Closure to save element in RDD in DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsTextFile(path)
    +
    +        return self.foreachRDD(saveAsTextFile)
    +
    +    def saveAsPickleFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a SequenceFile of serialized objects. The serializer
    +        used is L{pyspark.serializers.PickleSerializer}, default batch size
    +        is 10.
    +        """
    +
    +        def saveAsPickleFile(time, rdd):
    +            """
    +            Closure to save element in RDD in the DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsPickleFile(path)
    +
    +        return self.foreachRDD(saveAsPickleFile)
    +
    +    def transform(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, lambda t, a: func(a), True)
    +
    +    def transformWithTime(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, func, False)
    +
    +    def transformWith(self, func, other, keepSerializer=False):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream and 'other' DStream.
    +        """
    +        jfunc = RDDFunction(self.ctx, lambda t, a, b: func(a, b), self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonTransformed2DStream(self._jdstream.dstream(),
    +                                                          other._jdstream.dstream(), jfunc)
    +        jrdd_serializer = self._jrdd_deserializer if keepSerializer else self.ctx.serializer
    +        return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer)
    +
    +    def repartitions(self, numPartitions):
    +        """
    +        Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
    +        returned DStream has exactly numPartitions partitions.
    +        """
    +        return self.transform(lambda rdd: rdd.repartition(numPartitions))
    +
    +    @property
    +    def _slideDuration(self):
    +        """
    +        Return the slideDuration in seconds of this DStream
    +        """
    +        return self._jdstream.dstream().slideDuration().milliseconds() / 1000.0
    +
    +    def union(self, other):
    +        """
    +        Return a new DStream by unifying data of another DStream with this DStream.
    +        @param other Another DStream having the same interval (i.e., slideDuration) as this DStream.
    +        """
    +        if self._slideDuration != other._slideDuration:
    +            raise ValueError("the two DStream should have same slide duration")
    +        return self.transformWith(lambda a, b: a.union(b), other, True)
    +
    +    def cogroup(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'cogroup' between RDDs of `this`
    +        DStream and `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.cogroup(b, numPartitions), other)
    +
    +    def join(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.join(b, numPartitions), other)
    +
    +    def leftOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.leftOuterJoin(b, numPartitions), other)
    +
    +    def rightOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.rightOuterJoin(b, numPartitions), other)
    +
    +    def fullOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.fullOuterJoin(b, numPartitions), other)
    +
    +    def _jtime(self, timestamp):
    +        """ Convert datetime or unix_timestamp into Time
    +        """
    +        if isinstance(timestamp, datetime):
    +            timestamp = time.mktime(timestamp.timetuple())
    +        return self.ctx._jvm.Time(long(timestamp * 1000))
    +
    +    def slice(self, begin, end):
    +        """
    +        Return all the RDDs between 'begin' to 'end' (both included)
    +
    +        `begin`, `end` could be datetime.datetime() or unix_timestamp
    +        """
    +        jrdds = self._jdstream.slice(self._jtime(begin), self._jtime(end))
    +        return [RDD(jrdd, self.ctx, self._jrdd_deserializer) for jrdd in jrdds]
    +
    +    def _check_window(self, window, slide):
    --- End diff --
    
    Please add comments on what this method checks. Or just call it `_validate_window_params`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57240637
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20995/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18240810
  
    --- Diff: examples/src/main/python/streaming/network_wordcount.py ---
    @@ -0,0 +1,20 @@
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18132833
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,385 @@
    +#
    +# 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.
    +#
    +
    +"""
    +Unit tests for Python SparkStreaming; additional tests are implemented as doctests in
    +individual modules.
    +
    +Callback server is sometimes unstable sometimes, which cause error in test case.
    +But this is very rare case.
    +"""
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +
    +from pyspark.context import SparkContext
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import Seconds
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +
    +    timeout = 10  # seconds
    +
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        self.sc = SparkContext(appName=class_name)
    +        self.sc.setCheckpointDir("/tmp")
    +        # TODO: decrease duration to speed up tests
    +        self.ssc = StreamingContext(self.sc, duration=1)
    +
    +    def tearDown(self):
    +        self.ssc.stop()
    +
    +    @classmethod
    +    def tearDownClass(cls):
    +        # Make sure tp shutdown the callback server
    --- End diff --
    
    Sorry, this is my typo.
    
    ```
    # Make sure t'o' shutdown the callback server
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58051578
  
    @giwa @nchammas  -- yeah, we currently have jenkins hosted on a spotty, internal campus network.  we should be on the new, shiny, fast, redundant datacenter network by EOW.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58107711
  
    This is blocked by #2624 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57278200
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21029/consoleFull) for   PR 2538 at commit [`e00136b`](https://github.com/apache/spark/commit/e00136b3dfd330689d89e44006a49871b36a4825).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57868410
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/263/consoleFull) for   PR 2538 at commit [`52c535b`](https://github.com/apache/spark/commit/52c535b0696b3861222a7bd6608bb3f6f4db64c3).
     * This patch **passes** unit tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57129245
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20962/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57253697
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21008/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57427175
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21097/consoleFull) for   PR 2538 at commit [`6f0da2f`](https://github.com/apache/spark/commit/6f0da2fa486c2a580045a2e9e3133b6617875363).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57521322
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21145/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200872
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    --- End diff --
    
    I think I get it. You need a simple trait / interface (`PythonRDDFunction`) that is used as the interface for creating python function objects through py4j. And then you need to convert it to a `Function2` object (using `RDDFunction` class) so that it can be passed on to `DStream.foreachRDD`. However, I am not sure this warrants a separate class with a confusing name. How about static conversions added to `object PythonRDDFunction` like 
    ```
    object PythonRDDFunction {
       implicit def toFunction2(): Option[Function2[....]] = {
       }
    }
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193108
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,243 @@
    +#
    +# 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 ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD
    +from pyspark.serializers import UTF8Deserializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import RDDFunction
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    --- End diff --
    
    Can you add a bit more information in the docs, on how you add hack the py4j server. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57856814
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/263/consoleFull) for   PR 2538 at commit [`52c535b`](https://github.com/apache/spark/commit/52c535b0696b3861222a7bd6608bb3f6f4db64c3).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18554081
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,548 @@
    +#
    +# 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.
    +#
    +
    +import os
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +import tempfile
    +
    +from pyspark.context import SparkConf, SparkContext, RDD
    +from pyspark.streaming.context import StreamingContext
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +
    +    timeout = 10  # seconds
    +    duration = 1
    +
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        conf = SparkConf().set("spark.default.parallelism", 1)
    --- End diff --
    
    Why do we need to set this?  I wonder whether this could mask any bugs that might be exposed by having multiple partitions within a stage vs. a single partition.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58741172
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/349/consoleFull)**     for PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193173
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,243 @@
    +#
    +# 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 ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD
    +from pyspark.serializers import UTF8Deserializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import RDDFunction
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +            # self.port = self.server_socket.getsockname()[1]
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext represents the
    +    connection to a Spark cluster, and can be used to create L{DStream}s and
    +    broadcast variables on that cluster.
    +    """
    +
    +    def __init__(self, sparkContext, duration):
    +        """
    +        Create a new StreamingContext. At least the master and app name and duration
    +        should be set, either through the named parameters here or through C{conf}.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: seconds for SparkStreaming.
    +
    +        """
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._start_callback_server()
    +        self._jssc = self._initialize_context(self._sc, duration)
    +
    +    def _start_callback_server(self):
    +        gw = self._sc._gateway
    +        # getattr will fallback to JVM
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            gw._start_callback_server(gw._python_proxy_port)
    +            gw._python_proxy_port = gw._callback_server.port  # update port with real port
    +
    +    def _initialize_context(self, sc, duration):
    +        java_import(self._jvm, "org.apache.spark.streaming.*")
    +        java_import(self._jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(self._jvm, "org.apache.spark.streaming.api.python.*")
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @property
    +    def sparkContext(self):
    +        """
    +        Return SparkContext which is associated with this StreamingContext.
    +        """
    +        return self._sc
    +
    +    def start(self):
    +        """
    +        Start the execution of the streams.
    +        """
    +        self._jssc.start()
    +
    +    def awaitTermination(self, timeout=None):
    +        """
    +        Wait for the execution to stop.
    +        @param timeout: time to wait in seconds
    +        """
    +        if timeout is None:
    +            self._jssc.awaitTermination()
    +        else:
    +            self._jssc.awaitTermination(int(timeout * 1000))
    +
    +    def stop(self, stopSparkContext=True, stopGraceFully=False):
    +        """
    +        Stop the execution of the streams immediately (does not wait for all received data
    --- End diff --
    
    Wrong doc string. The Scala doc string is 
    /**
       * Stop the execution of the streams, with option of ensuring all received data
       * has been processed.
       * @param stopSparkContext Stop the associated SparkContext or not
       * @param stopGracefully Stop gracefully by waiting for the processing of all
       *                       received data to be completed
       */


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58733200
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58736402
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21612/consoleFull)**     for PR 2538 at commit [`3e2492b`](https://github.com/apache/spark/commit/3e2492b9b95e0cc0e3427265f71f069000cc43f7)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58726793
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21604/consoleFull) for   PR 2538 at commit [`3e2492b`](https://github.com/apache/spark/commit/3e2492b9b95e0cc0e3427265f71f069000cc43f7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18372715
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,305 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    --- End diff --
    
    For consistency's sake, we might just want to copy and adapt the Scala versions of these docstrings.   e.g. 
    
    ```
    @param batchDuration the time interval at which streaming data will be divided into batches
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2538


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57815905
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21251/consoleFull) for   PR 2538 at commit [`52c535b`](https://github.com/apache/spark/commit/52c535b0696b3861222a7bd6608bb3f6f4db64c3).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194179
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,473 @@
    +#
    +# 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.
    +#
    +
    +import os
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +import tempfile
    +
    +from pyspark.context import SparkContext
    +from pyspark.streaming.context import StreamingContext
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +
    +    timeout = 10  # seconds
    +
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        self.sc = SparkContext(appName=class_name)
    +        self.sc.setCheckpointDir("/tmp")
    +        # TODO: decrease duration to speed up tests
    +        self.ssc = StreamingContext(self.sc, duration=1)
    +
    +    def tearDown(self):
    +        self.ssc.stop()
    +
    +    def _test_func(self, input, func, expected, sort=False, input2=None):
    +        """
    +        @param input: dataset for the test. This should be list of lists.
    +        @param func: wrapped function. This function should return PythonDStream object.
    +        @param expected: expected output for this testcase.
    +        """
    +        input_stream = self.ssc.queueStream(input)
    +        input_stream2 = self.ssc.queueStream(input2) if input2 is not None else None
    +        # Apply test function to stream.
    +        if input2:
    +            stream = func(input_stream, input_stream2)
    +        else:
    +            stream = func(input_stream)
    +
    +        result = stream._collect()
    +        self.ssc.start()
    +
    +        start_time = time.time()
    +        # Loop until get the expected the number of the result from the stream.
    +        while True:
    +            current_time = time.time()
    +            # Check time out.
    +            if (current_time - start_time) > self.timeout:
    +                break
    +            # StreamingContext.awaitTermination is not used to wait because
    +            # if py4j server is called every 50 milliseconds, it gets an error.
    +            time.sleep(0.05)
    +            # Check if the output is the same length of expected output.
    +            if len(expected) == len(result):
    +                break
    +        if sort:
    +            self._sort_result_based_on_key(result)
    +            self._sort_result_based_on_key(expected)
    +        self.assertEqual(expected, result)
    +
    +    def _sort_result_based_on_key(self, outputs):
    +        """Sort the list based on first value."""
    +        for output in outputs:
    +            output.sort(key=lambda x: x[0])
    +
    +
    +class TestBasicOperations(PySparkStreamingTestCase):
    +
    +    def test_take(self):
    +        input = [range(i) for i in range(3)]
    +        dstream = self.ssc.queueStream(input)
    +        self.assertEqual([0, 0, 1], dstream._take(3))
    +
    +    def test_first(self):
    +        input = [range(10)]
    +        dstream = self.ssc.queueStream(input)
    +        self.assertEqual(0, dstream._first())
    +
    +    def test_map(self):
    +        """Basic operation test for DStream.map."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.map(str)
    +        expected = map(lambda x: map(str, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_flatMap(self):
    +        """Basic operation test for DStream.faltMap."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.flatMap(lambda x: (x, x * 2))
    +        expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))),
    +                       input)
    +        self._test_func(input, func, expected)
    +
    +    def test_filter(self):
    +        """Basic operation test for DStream.filter."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.filter(lambda x: x % 2 == 0)
    +        expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_count(self):
    +        """Basic operation test for DStream.count."""
    +        input = [range(5), range(10), range(20)]
    +
    +        def func(dstream):
    +            return dstream.count()
    +        expected = map(lambda x: [len(x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduce(self):
    +        """Basic operation test for DStream.reduce."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.reduce(operator.add)
    +        expected = map(lambda x: [reduce(operator.add, x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduceByKey(self):
    +        """Basic operation test for DStream.reduceByKey."""
    +        input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)],
    +                 [("", 1), ("", 1), ("", 1), ("", 1)],
    +                 [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]]
    +
    +        def func(dstream):
    +            return dstream.reduceByKey(operator.add)
    +        expected = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_mapValues(self):
    +        """Basic operation test for DStream.mapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 2), (3, 3)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.mapValues(lambda x: x + 10)
    +        expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)],
    +                    [("", 14), (1, 11), (2, 12), (3, 13)],
    +                    [(1, 11), (2, 11), (3, 11), (4, 11)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_flatMapValues(self):
    +        """Basic operation test for DStream.flatMapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 1), (3, 1)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.flatMapValues(lambda x: (x, x + 10))
    +        expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12),
    +                     ("c", 1), ("c", 11), ("d", 1), ("d", 11)],
    +                    [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)],
    +                    [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]]
    +        self._test_func(input, func, expected)
    +
    +    def test_glom(self):
    +        """Basic operation test for DStream.glom."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            return dstream.glom()
    +        expected = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_mapPartitions(self):
    +        """Basic operation test for DStream.mapPartitions."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            def f(iterator):
    +                yield sum(iterator)
    +            return dstream.mapPartitions(f)
    +        expected = [[3, 7], [11, 15], [19, 23]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_countByValue(self):
    +        """Basic operation test for DStream.countByValue."""
    +        input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]]
    +
    +        def func(dstream):
    +            return dstream.countByValue()
    +        expected = [[4], [4], [3]]
    +        self._test_func(input, func, expected)
    +
    +    def test_groupByKey(self):
    +        """Basic operation test for DStream.groupByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            return dstream.groupByKey().mapValues(list)
    +
    +        expected = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])],
    +                    [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])],
    +                    [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_combineByKey(self):
    +        """Basic operation test for DStream.combineByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            def add(a, b):
    +                return a + str(b)
    +            return dstream.combineByKey(str, add, add)
    +        expected = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")],
    +                    [(1, "111"), (2, "11"), (3, "1")],
    +                    [("a", "11"), ("b", "1"), ("", "111")]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_repartition(self):
    +        input = [range(1, 5), range(5, 9)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            return dstream.repartitions(1).glom()
    +        expected = [[[1, 2, 3, 4]], [[5, 6, 7, 8]]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_union(self):
    +        input1 = [range(3), range(5), range(6)]
    +        input2 = [range(3, 6), range(5, 6)]
    +
    +        def func(d1, d2):
    +            return d1.union(d2)
    +
    +        expected = [range(6), range(6), range(6)]
    +        self._test_func(input1, func, expected, input2=input2)
    +
    +    def test_cogroup(self):
    +        input = [[(1, 1), (2, 1), (3, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1)]]
    +        input2 = [[(1, 2)],
    +                  [(4, 1)],
    +                  [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 2)]]
    +
    +        def func(d1, d2):
    +            return d1.cogroup(d2).mapValues(lambda vs: tuple(map(list, vs)))
    +
    +        expected = [[(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))],
    +                    [(1, ([1, 1, 1], [])), (2, ([1], [])), (4, ([], [1]))],
    +                    [("a", ([1, 1], [1, 1])), ("b", ([1], [1])), ("", ([1, 1], [1, 2]))]]
    +        self._test_func(input, func, expected, sort=True, input2=input2)
    +
    +    def test_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.join(b)
    +
    +        expected = [[('b', (2, 3))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_left_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.leftOuterJoin(b)
    +
    +        expected = [[('a', (1, None)), ('b', (2, 3))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_right_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.rightOuterJoin(b)
    +
    +        expected = [[('b', (2, 3)), ('c', (None, 4))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_full_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.fullOuterJoin(b)
    +
    +        expected = [[('a', (1, None)), ('b', (2, 3)), ('c', (None, 4))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +
    +class TestWindowFunctions(PySparkStreamingTestCase):
    +
    +    timeout = 20
    +
    +    def test_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5)]
    +
    +        def func(dstream):
    +            return dstream.window(3, 1).count()
    +
    +        expected = [[1], [3], [6], [9], [12], [9], [5]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5)]
    +
    +        def func(dstream):
    +            return dstream.countByWindow(3, 1)
    +
    +        expected = [[1], [3], [6], [9], [12], [9], [5]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_window_large(self):
    +        input = [range(1), range(2), range(3), range(4), range(5), range(6)]
    +
    +        def func(dstream):
    +            return dstream.countByWindow(5, 1)
    +
    +        expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_value_and_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5), range(6)]
    +
    +        def func(dstream):
    +            return dstream.countByValueAndWindow(5, 1)
    +
    +        expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]]
    +        self._test_func(input, func, expected)
    +
    +    def test_group_by_key_and_window(self):
    +        input = [[('a', i)] for i in range(5)]
    +
    +        def func(dstream):
    +            return dstream.groupByKeyAndWindow(3, 1).mapValues(list)
    +
    +        expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])],
    +                    [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]]
    +        self._test_func(input, func, expected)
    +
    +    def test_reduce_by_invalid_window(self):
    +        input1 = [range(3), range(5), range(1), range(6)]
    +        d1 = self.ssc.queueStream(input1)
    +        self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 0.1, 0.1))
    +        self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 1, 0.1))
    +
    +    def update_state_by_key(self):
    --- End diff --
    
    This is not a window operation, and hence should go into basic operations suite. This will be consistent with Scala testsuites.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047297
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20909/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58274202
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21413/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194347
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    --- End diff --
    
    Please add Scala/Java docs for these classes (even if they are internal classes). Helps one understand the code, which is anyways a little complex for PySpark stuff.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18202894
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18201096
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pyfunc: PythonRDDFunction){
    +    val func = new RDDFunction(pyfunc)
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    +    val func = new RDDFunction(pyfunc)
    +    ssc.transform(jdsteams, func)
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    + * this can reduce the Python callbacks.
    + */
    +private[spark]
    --- End diff --
    
    Why `private[spark]` and not `private[python]`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194443
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pyfunc: PythonRDDFunction){
    +    val func = new RDDFunction(pyfunc)
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    +    val func = new RDDFunction(pyfunc)
    +    ssc.transform(jdsteams, func)
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    + * this can reduce the Python callbacks.
    + */
    +private[spark]
    +class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,
    +                                var reuse: Boolean = false)
    +  extends PythonDStream(parent) {
    +
    +  val func = new RDDFunction(pfunc)
    +  var lastResult: PythonRDD = _
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val rdd1 = parent.getOrCompute(validTime)
    --- End diff --
    
    Please add some comments on what this method does, especially regarding reuse and all. Also, how is `lastResult` used?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18070136
  
    --- Diff: python/pyspark/streaming/jtime.py ---
    @@ -0,0 +1,135 @@
    +#
    +# 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 pyspark.streaming.duration import Duration
    +
    +"""
    +The name of this file, time is not a good naming for python
    --- End diff --
    
    Take a look at https://github.com/apache/spark/pull/1628#issuecomment-50663192 for a discussion of a similar issue. It's possible to name this time.py without any conflicts; look at libraries like numpy, where you have numpy.random, to see that this is possible.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57230490
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/196/consoleFull) for   PR 2538 at commit [`19797f9`](https://github.com/apache/spark/commit/19797f9fc9b062ee30746c184ad432192ca5e19a).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56994595
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20859/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58695306
  
    @davies I see. This would be huge PR. It makes sense to test more before merging this PR. Thank you!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58722102
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18235197
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  // callback in Python
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    PythonDStream.some(pfunc.call(time.milliseconds, List(PythonDStream.wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(PythonDStream.wrapRDD(rdd), PythonDStream.wrapRDD(rdd2)).asJava
    +    PythonDStream.some(pfunc.call(time.milliseconds, rdds))
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +
    +/**
    + * Base class for PythonDStream with some common methods
    + */
    +private[python]
    +abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * Helper functions
    + */
    +private[python] object PythonDStream {
    +
    +  // convert Option[RDD[_]] to JavaRDD, handle null gracefully
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    --- End diff --
    
    Isnt this just `rdd.map(JavaRDD.fromRDD).orNull` ? Why not use this directly? Sorry should have observed that earlier :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57957378
  
    Why was this error happened in jenkins? Is this because of many commits?
    
    The error said
    ```
    Fetching upstream changes from https://github.com/apache/spark.git
     > git --version # timeout=10
     > git fetch --tags --progress https://github.com/apache/spark.git +refs/pull/*:refs/remotes/origin/pr/*
    ERROR: Timeout after 10 minutes
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58047057
  
    > Why was this error happened in jenkins? Is this because of many commits?
    
    No, I believe this is related to Jenkins being hosted on some servers with spotty connections to the outside world. @shaneknapp is working on moving Jenkins to a proper datacenter, after which these problems should hopefully go away.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18235396
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  // callback in Python
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    PythonDStream.some(pfunc.call(time.milliseconds, List(PythonDStream.wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(PythonDStream.wrapRDD(rdd), PythonDStream.wrapRDD(rdd2)).asJava
    +    PythonDStream.some(pfunc.call(time.milliseconds, rdds))
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +
    +/**
    + * Base class for PythonDStream with some common methods
    + */
    +private[python]
    +abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * Helper functions
    + */
    +private[python] object PythonDStream {
    +
    +  // convert Option[RDD[_]] to JavaRDD, handle null gracefully
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  // convert JavaRDD to Option[RDD[Array[Byte]]] to , handle null gracefully
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pfunc: PythonRDDFunction){
    +    val func = new RDDFunction((pfunc))
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    --- End diff --
    
    The style is wrong I think, there should be space between `:` and `JavaDStream`. How about
    ```
    def callTransform(
          ssc: JavaStreamingContext, 
          jdsteams: JList[JavaDStream[_]], 
          pyfunc: PythonRDDFunction
       ): JavaDStream[Array[Byte]] = {
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58291439
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/283/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58726799
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21604/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57724027
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/257/consoleFull)**     for PR 2538 at commit [`d05871e`](https://github.com/apache/spark/commit/d05871e912ee4828a4ac68a6a0ceed0454e44722)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58709000
  
    I am playing around with this right now. Will merge after I do some basic testing. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18554420
  
    --- Diff: python/pyspark/streaming/util.py ---
    @@ -0,0 +1,113 @@
    +#
    +# 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.
    +#
    +
    +import time
    +from datetime import datetime
    +import traceback
    +
    +from pyspark import SparkContext, RDD
    +
    +
    +class TransformFunction(object):
    +    """
    +    This class is for py4j callback.
    --- End diff --
    
    This `TransformFunction` (and its Java counterpart) is one of the most complex parts of the PySpark Streaming code, so it would help to have some additional documentation explaining how it works.
    
    To confirm my understanding:
    
    > This class wraps a function `RDD[X] -> RDD[Y]` that was passed to `DStream.transform()`, allowing it to be called from Java via Py4J's callback server.  Java calls this function with a sequence of JavaRDDs and this function returns a single JavaRDD pointer back to Java.
    
    Is that an accurate summary?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047501
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20910/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18235125
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  // callback in Python
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    PythonDStream.some(pfunc.call(time.milliseconds, List(PythonDStream.wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(PythonDStream.wrapRDD(rdd), PythonDStream.wrapRDD(rdd2)).asJava
    +    PythonDStream.some(pfunc.call(time.milliseconds, rdds))
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +
    +/**
    + * Base class for PythonDStream with some common methods
    + */
    +private[python]
    +abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * Helper functions
    + */
    +private[python] object PythonDStream {
    +
    +  // convert Option[RDD[_]] to JavaRDD, handle null gracefully
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  // convert JavaRDD to Option[RDD[Array[Byte]]] to , handle null gracefully
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    --- End diff --
    
    Isnt this function essentially, `Option(jrdd).map { _.rdd }` ? It might be worth using that directly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047298
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20909/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56928665
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20846/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57253684
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21008/consoleFull) for   PR 2538 at commit [`338580a`](https://github.com/apache/spark/commit/338580a7aa39fcf8beedefdc7000b906a1028c84).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194255
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -413,7 +413,7 @@ class StreamingContext private[streaming] (
           dstreams: Seq[DStream[_]],
           transformFunc: (Seq[RDD[_]], Time) => RDD[T]
         ): DStream[T] = {
    -    new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc))
    +    new TransformedDStream[T](dstreams, transformFunc)
    --- End diff --
    
    Why this change?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57656636
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21203/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58282747
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21425/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18378110
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,305 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    --- End diff --
    
    Yeah, I had mentioned that earlier. We shoudl keep the documentation consistent.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56906051
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20833/consoleFull) for   PR 2538 at commit [`bd27874`](https://github.com/apache/spark/commit/bd278742c59b9d9cd917cd5fd4ca8262aad1f64c).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58284764
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/283/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18251679
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.io.{ObjectInputStream, ObjectOutputStream}
    +import java.lang.reflect.Proxy
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + * TODO: support checkpoint
    + */
    +private[python] class RDDFunction(@transient var pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    Option(pfunc.call(time.milliseconds, List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava)).map(_.rdd)
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull, rdd2.map(JavaRDD.fromRDD(_)).orNull).asJava
    +    Option(pfunc.call(time.milliseconds, rdds)).map(_.rdd)
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +
    +  private def writeObject(out: ObjectOutputStream): Unit = {
    +    assert(PythonDStream.serializer != null, "Serializer has not been registered!")
    +    val bytes = PythonDStream.serializer.serialize(pfunc)
    +    out.writeInt(bytes.length)
    +    out.write(bytes)
    +  }
    +
    +  private def readObject(in: ObjectInputStream): Unit = {
    +    assert(PythonDStream.serializer != null, "Serializer has not been registered!")
    +    val length = in.readInt()
    +    val bytes = new Array[Byte](length)
    +    in.readFully(bytes)
    +    pfunc = PythonDStream.serializer.deserialize(bytes)
    +  }
    +}
    +
    +/**
    + * Inferface for Python Serializer to serialize PythonRDDFunction
    + */
    +private[python] trait PythonRDDFunctionSerializer {
    +  def dumps(id: String): Array[Byte]  //
    +  def loads(bytes: Array[Byte]): PythonRDDFunction
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunctionSerializer
    + */
    +private[python] class RDDFunctionSerializer(pser: PythonRDDFunctionSerializer) {
    +  def serialize(func: PythonRDDFunction): Array[Byte] = {
    +    // get the id of PythonRDDFunction in py4j
    +    val h = Proxy.getInvocationHandler(func.asInstanceOf[Proxy])
    +    val f = h.getClass().getDeclaredField("id");
    +    f.setAccessible(true);
    +    val id = f.get(h).asInstanceOf[String];
    +    pser.dumps(id)
    +  }
    +
    +  def deserialize(bytes: Array[Byte]): PythonRDDFunction = {
    +    pser.loads(bytes)
    +  }
    +}
    +
    +/**
    + * Helper functions
    + */
    +private[python] object PythonDStream {
    +
    +  // A serializer in Python, used to serialize PythonRDDFunction
    +  var serializer: RDDFunctionSerializer = _
    +
    +  // Register a serializer from Python, should be called during initialization
    +  def registerSerializer(ser: PythonRDDFunctionSerializer) = {
    +    serializer = new RDDFunctionSerializer(ser)
    +  }
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pfunc: PythonRDDFunction) {
    +    val func = new RDDFunction((pfunc))
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Base class for PythonDStream with some common methods
    + */
    +private[python]
    +abstract class PythonDStream(parent: DStream[_], @transient pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If `reuse` is true and the result of the `func` is an PythonRDD, then it will cache it
    + * as an template for future use, this can reduce the Python callbacks.
    + */
    +private[python]
    +class PythonTransformedDStream (parent: DStream[_], @transient pfunc: PythonRDDFunction,
    +                                var reuse: Boolean = false)
    +  extends PythonDStream(parent, pfunc) {
    +
    +  // rdd returned by func
    +  var lastResult: PythonRDD = _
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val rdd = parent.getOrCompute(validTime)
    +    if (rdd.isEmpty) {
    +      return None
    +    }
    +    if (reuse && lastResult != null) {
    +      // use the previous result as the template to generate new RDD
    +      Some(lastResult.copyTo(rdd.get))
    +    } else {
    +      val r = func(rdd, validTime)
    +      if (reuse && r.isDefined && lastResult == null) {
    +        // try to use the result as a template
    +        r.get match {
    +          case pyrdd: PythonRDD =>
    +            if (pyrdd.parent(0) == rdd) {
    +              // only one PythonRDD
    +              lastResult = pyrdd
    +            } else {
    +              // maybe have multiple stages, don't check it anymore
    +              reuse = false
    +            }
    +        }
    +      }
    +      r
    +    }
    +  }
    +}
    +
    +/**
    + * Transformed from two DStreams in Python.
    + */
    +private[python]
    +class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],
    +                                @transient pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  override def dependencies = List(parent, parent2)
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    func(parent.getOrCompute(validTime), parent2.getOrCompute(validTime), validTime)
    +  }
    +
    +  val asJavaDStream = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * similar to StateDStream
    + */
    +private[python]
    +class PythonStateDStream(parent: DStream[Array[Byte]], @transient reduceFunc: PythonRDDFunction)
    +  extends PythonDStream(parent, reduceFunc) {
    +
    +  super.persist(StorageLevel.MEMORY_ONLY)
    +  override val mustCheckpoint = true
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val lastState = getOrCompute(validTime - slideDuration)
    +    val rdd = parent.getOrCompute(validTime)
    +    if (rdd.isDefined) {
    +      func(lastState, rdd, validTime)
    +    } else {
    +      lastState
    +    }
    +  }
    +}
    +
    +/**
    + * similar to ReducedWindowedDStream
    + */
    +private[python]
    +class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],
    --- End diff --
    
    Incorrect scala style. See http://docs.scala-lang.org/style/declarations.html


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57383332
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21066/consoleFull) for   PR 2538 at commit [`fa7261b`](https://github.com/apache/spark/commit/fa7261b5610a02fe725f975fada995d37234f615).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57240622
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20995/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18234439
  
    --- Diff: examples/src/main/python/streaming/network_wordcount.py ---
    @@ -0,0 +1,20 @@
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    --- End diff --
    
    i forgot to mention, can you add instruction on how to run the example (along with nc, etc.) as doc comments? See the comments in the scala / java NetworkWordCount.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193637
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        def func(rdd):
    +            return rdd.combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions)
    --- End diff --
    
    There is a slight inconsistency with this approach of the setting the default of number of partitions. The Scala/Java DStream API uses an older way of determining the number of partitions, which uses the default parallelism configuration (see code in PairDStreamFunctions). But this uses the smarter default partitions based on the number of partitions of the pre-shuffle RDD. There maybe significant performance difference. It maybe a good idea to keep things consistent, and consider improving this in a different JIRA+PR. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193243
  
    --- Diff: python/pyspark/java_gateway.py ---
    @@ -23,6 +23,7 @@
     import platform
     from subprocess import Popen, PIPE
     from threading import Thread
    +
    --- End diff --
    
    Is this space intended?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047597
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20911/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57535831
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21151/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57020046
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20876/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200031
  
    --- Diff: bin/pyspark ---
    @@ -87,11 +87,7 @@ export PYSPARK_SUBMIT_ARGS
     if [[ -n "$SPARK_TESTING" ]]; then
       unset YARN_CONF_DIR
       unset HADOOP_CONF_DIR
    -  if [[ -n "$PYSPARK_DOC_TEST" ]]; then
    -    exec "$PYSPARK_PYTHON" -m doctest $1
    -  else
    -    exec "$PYSPARK_PYTHON" $1
    -  fi
    +  exec "$PYSPARK_PYTHON" $1
    --- End diff --
    
    it's not necessary, it's part of refactor run-tests, do we need to create a separate PR for this?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58115560
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21347/Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58738838
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/349/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58739234
  
    @tdas it's my mistake, the updateStateByKey() was used in another tests, it's fixed 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58691755
  
    @giwa No, it's under testing/QA


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18378095
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,305 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    --- End diff --
    
    I agree. Good catch.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57230596
  
    @davies 
    This error is interesting. I also doubt start and stop StreamingContext. I changed the order of execution of tests. I run TestWindowFunctions first. However, I sometimes got the same error.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56926666
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20843/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57249603
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21008/consoleFull) for   PR 2538 at commit [`338580a`](https://github.com/apache/spark/commit/338580a7aa39fcf8beedefdc7000b906a1028c84).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18080721
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,321 @@
    +#
    +# 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.
    +#
    +
    +"""
    +Unit tests for Python SparkStreaming; additional tests are implemented as doctests in
    +individual modules.
    +
    +Callback server is sometimes unstable sometimes, which cause error in test case.
    +But this is very rare case.
    +"""
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +
    +from pyspark.context import SparkContext
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import Seconds
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        self.sc = SparkContext(appName=class_name)
    +        self.ssc = StreamingContext(self.sc, duration=Seconds(1))
    +
    +    def tearDown(self):
    +        # Do not call pyspark.streaming.context.StreamingContext.stop directly because
    +        # we do not wait to shutdown py4j client.
    --- End diff --
    
    We do not need this comments because StreaminigContext.stop does not stop call back server.
    
    ```
            # Do not call pyspark.streaming.context.StreamingContext.stop directly because
            # we do not wait to shutdown py4j client.
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57348522
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21042/consoleFull) for   PR 2538 at commit [`9a16bd1`](https://github.com/apache/spark/commit/9a16bd1bdce5b66ff3701aeb94b77d94e8b0a521).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047595
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20911/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194241
  
    --- Diff: python/run-tests ---
    @@ -60,29 +93,10 @@ fi
     echo "Testing with Python version:"
     $PYSPARK_PYTHON --version
     
    -run_test "pyspark/rdd.py"
    -run_test "pyspark/context.py"
    -run_test "pyspark/conf.py"
    -run_test "pyspark/sql.py"
    -# These tests are included in the module-level docs, and so must
    -# be handled on a higher level rather than within the python file.
    -export PYSPARK_DOC_TEST=1
    -run_test "pyspark/broadcast.py"
    -run_test "pyspark/accumulators.py"
    -run_test "pyspark/serializers.py"
    -unset PYSPARK_DOC_TEST
    -run_test "pyspark/shuffle.py"
    -run_test "pyspark/tests.py"
    -run_test "pyspark/mllib/classification.py"
    -run_test "pyspark/mllib/clustering.py"
    -run_test "pyspark/mllib/linalg.py"
    -run_test "pyspark/mllib/random.py"
    -run_test "pyspark/mllib/recommendation.py"
    -run_test "pyspark/mllib/regression.py"
    -run_test "pyspark/mllib/stat.py"
    -run_test "pyspark/mllib/tests.py"
    -run_test "pyspark/mllib/tree.py"
    -run_test "pyspark/mllib/util.py"
    +#run_core_tests
    --- End diff --
    
    These should not be commented out finally, right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58739136
  
    @tdas The failure looked wired, updater() take exactly two arguments, let's test it again.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200124
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -256,3 +256,8 @@ def _start_update_server():
         thread.daemon = True
         thread.start()
         return server
    +
    +
    +if __name__ == "__main__":
    --- End diff --
    
    it's part of run-tests refactor


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57226638
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20997/consoleFull) for   PR 2538 at commit [`19797f9`](https://github.com/apache/spark/commit/19797f9fc9b062ee30746c184ad432192ca5e19a).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57383495
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21066/consoleFull) for   PR 2538 at commit [`fa7261b`](https://github.com/apache/spark/commit/fa7261b5610a02fe725f975fada995d37234f615).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class RDDFunction(object):`
      * `class RDDFunctionSerializer(object):`
      * `abstract class PythonDStream(parent: DStream[_], @transient pfunc: PythonRDDFunction)`
      * `class PythonTransformedDStream (parent: DStream[_], @transient pfunc: PythonRDDFunction,`
      * `class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],`
      * `class PythonStateDStream(parent: DStream[Array[Byte]], @transient reduceFunc: PythonRDDFunction)`
      * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18203052
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pyfunc: PythonRDDFunction){
    +    val func = new RDDFunction(pyfunc)
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    +    val func = new RDDFunction(pyfunc)
    +    ssc.transform(jdsteams, func)
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    + * this can reduce the Python callbacks.
    + */
    +private[spark]
    +class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,
    +                                var reuse: Boolean = false)
    +  extends PythonDStream(parent) {
    +
    +  val func = new RDDFunction(pfunc)
    +  var lastResult: PythonRDD = _
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val rdd1 = parent.getOrCompute(validTime)
    +    if (rdd1.isEmpty) {
    +      return None
    +    }
    +    if (reuse && lastResult != null) {
    +      Some(lastResult.copyTo(rdd1.get))
    +    } else {
    +      val r = func(rdd1, validTime)
    +      if (reuse && r.isDefined && lastResult == null) {
    +        r.get match {
    +          case rdd: PythonRDD =>
    +            if (rdd.parent(0) == rdd1) {
    +              // only one PythonRDD
    +              lastResult = rdd
    +            } else {
    +              // may have multiple stages
    +              reuse = false
    +            }
    +        }
    +      }
    +      r
    +    }
    +  }
    +}
    +
    +/**
    + * Transformed from two DStreams in Python.
    + */
    +private[spark]
    +class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],
    +                                pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  override def dependencies = List(parent, parent2)
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    func(parent.getOrCompute(validTime), parent2.getOrCompute(validTime), validTime)
    +  }
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * similar to StateDStream
    + */
    +private[spark]
    +class PythonStateDStream(parent: DStream[Array[Byte]], preduceFunc: PythonRDDFunction)
    +  extends PythonDStream(parent) {
    +
    +  val reduceFunc = new RDDFunction(preduceFunc)
    +
    +  super.persist(StorageLevel.MEMORY_ONLY)
    +  override val mustCheckpoint = true
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val lastState = getOrCompute(validTime - slideDuration)
    +    val rdd = parent.getOrCompute(validTime)
    +    if (rdd.isDefined) {
    +      reduceFunc(lastState, rdd, validTime)
    +    } else {
    +      lastState
    +    }
    +  }
    +}
    +
    +/**
    + * similar to ReducedWindowedDStream
    + */
    +private[spark]
    +class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],
    --- End diff --
    
    it's called in Python, should private[spark] also work?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57722903
  
    @JoshRosen Agreed, but this PR is so huge now, we can do later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57694687
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21219/consoleFull) for   PR 2538 at commit [`d05871e`](https://github.com/apache/spark/commit/d05871e912ee4828a4ac68a6a0ceed0454e44722).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56906115
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20833/consoleFull) for   PR 2538 at commit [`bd27874`](https://github.com/apache/spark/commit/bd278742c59b9d9cd917cd5fd4ca8262aad1f64c).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedRDD(DStream):`
      * `class Duration(object):`
      * `class Time(object):`
      * `class RDDFunction(object):`
      * `trait PythonRDDFunction `
      * `class PythonTransformedDStream (parent: DStream[_], parent2: DStream[_], func: PythonRDDFunction,`
      * `class PythonForeachDStream(`
      * `class PythonDataInputStream(`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57279516
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21031/consoleFull) for   PR 2538 at commit [`b98d63f`](https://github.com/apache/spark/commit/b98d63fbde10f20a42e1e6e0f34f45736b802772).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57279309
  
    @giwa After fixing the problem of increasing partitions (it will increase performance problem), the tests run very stable 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57761606
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21241/consoleFull) for   PR 2538 at commit [`d817de8`](https://github.com/apache/spark/commit/d817de8bae1fa0ddafd257a8d1d68a025a8cde66).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18234505
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -114,6 +114,9 @@ def __ne__(self, other):
         def __repr__(self):
             return "<%s object>" % self.__class__.__name__
     
    +    def __hash__(self):
    +        return hash(str(self))
    --- End diff --
    
    Gotcha.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57079950
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20932/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18201085
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    --- End diff --
    
    Why `private[spark]` and not `private[python]` ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57427252
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21097/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18202866
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -114,6 +114,9 @@ def __ne__(self, other):
         def __repr__(self):
             return "<%s object>" % self.__class__.__name__
     
    +    def __hash__(self):
    +        return hash(str(self))
    --- End diff --
    
    This is necessary, we need to check the serializers of dstreams.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57279408
  
    @tdas I should have addressed all your comments (or leave comment), please take another look.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18070171
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,141 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList}
    +
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.streaming.{Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function
    + */
    +trait PythonRDDFunction {
    +  def call(rdd: JavaRDD[_], rdd2: JavaRDD[_], time: Long): JavaRDD[Array[Byte]]
    +}
    +
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    + * this can reduce the Python callbacks.
    + *
    + * @param parent
    --- End diff --
    
    Can delete this Scaladoc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18073938
  
    --- Diff: python/pyspark/streaming/jtime.py ---
    @@ -0,0 +1,135 @@
    +#
    +# 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 pyspark.streaming.duration import Duration
    +
    +"""
    +The name of this file, time is not a good naming for python
    --- End diff --
    
    That will introduce some unnecessary troubles, also I'd like to remove this module later.
    
    The hacking for random will be broke sometimes, is user modify sys.path, I will try to fix that. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57854801
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18234833
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala ---
    @@ -413,7 +413,7 @@ class StreamingContext private[streaming] (
           dstreams: Seq[DStream[_]],
           transformFunc: (Seq[RDD[_]], Time) => RDD[T]
         ): DStream[T] = {
    -    new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc))
    +    new TransformedDStream[T](dstreams, transformFunc)
    --- End diff --
    
    But the callback function needs to be serializable, otherwise checkpointing of the DStream DAG will fail. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57761610
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21241/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18557162
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,548 @@
    +#
    +# 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.
    +#
    +
    +import os
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +import tempfile
    +
    +from pyspark.context import SparkConf, SparkContext, RDD
    +from pyspark.streaming.context import StreamingContext
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +
    +    timeout = 10  # seconds
    +    duration = 1
    +
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        conf = SparkConf().set("spark.default.parallelism", 1)
    --- End diff --
    
    Setting this to minimize the runtime in each iteration, or the tests will be flaky if one batch can not finish in the batch window, we can increase this after improve the latency.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57204099
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20981/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58273858
  
    Lets if any of the tests there are flaky or not.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18373236
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,305 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    +        """
    +
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._jssc = jssc or self._initialize_context(self._sc, duration)
    +
    +    def _initialize_context(self, sc, duration):
    +        self._ensure_initialized()
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @classmethod
    +    def _ensure_initialized(cls):
    +        SparkContext._ensure_initialized()
    +        gw = SparkContext._gateway
    +        # start callback server
    +        # getattr will fallback to JVM
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            gw._start_callback_server(gw._python_proxy_port)
    +
    +        java_import(gw.jvm, "org.apache.spark.streaming.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.python.*")
    +        # register serializer for TransformFunction
    +        # it happens before creating SparkContext when loading from checkpointing
    +        cls._transformerSerializer = TransformFunctionSerializer(
    +            SparkContext._active_spark_context, CloudPickleSerializer(), gw)
    +
    +    @classmethod
    +    def getOrCreate(cls, path, setupFunc):
    +        """
    +        Get the StreamingContext from checkpoint file at `path`, or setup
    --- End diff --
    
    Here, too, we might want to copy the Scala docs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58774562
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/352/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57510600
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57495542
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21128/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57286041
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21031/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57636755
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/256/consoleFull) for   PR 2538 at commit [`54bd92b`](https://github.com/apache/spark/commit/54bd92b5800ce9165e53289c44603e6a89c5ed75).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047499
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20910/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58795386
  
    Thank you for merging this! I appreciate to you, @tdas , to give me many advices. Thank you, @davies  to refactor code a lot. I learned many thing from your commits. And also thank you to @JoshRosen  to give us the insight and discussion.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18555184
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.io.{ObjectInputStream, ObjectOutputStream}
    +import java.lang.reflect.Proxy
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.language.existentials
    +
    +import py4j.GatewayServer
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function which is used to transform RDDs
    + */
    +private[python] trait PythonTransformFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Interface for Python Serializer to serialize PythonTransformFunction
    + */
    +private[python] trait PythonTransformFunctionSerializer {
    +  def dumps(id: String): Array[Byte]
    +  def loads(bytes: Array[Byte]): PythonTransformFunction
    +}
    +
    +/**
    + * Wrapper for PythonTransformFunction
    + */
    +private[python] class TransformFunction(@transient var pfunc: PythonTransformFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    Option(pfunc.call(time.milliseconds, List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava))
    +      .map(_.rdd)
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull, rdd2.map(JavaRDD.fromRDD(_)).orNull).asJava
    +    Option(pfunc.call(time.milliseconds, rdds)).map(_.rdd)
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +
    +  private def writeObject(out: ObjectOutputStream): Unit = {
    +    val bytes = PythonTransformFunctionSerializer.serialize(pfunc)
    +    out.writeInt(bytes.length)
    +    out.write(bytes)
    +  }
    +
    +  private def readObject(in: ObjectInputStream): Unit = {
    +    val length = in.readInt()
    +    val bytes = new Array[Byte](length)
    +    in.readFully(bytes)
    +    pfunc = PythonTransformFunctionSerializer.deserialize(bytes)
    +  }
    +}
    +
    +/**
    + * Helpers for PythonTransformFunctionSerializer
    + */
    +private[python] object PythonTransformFunctionSerializer {
    +
    +  /**
    +   * A serializer in Python, used to serialize PythonTransformFunction
    +    */
    +  private var serializer: PythonTransformFunctionSerializer = _
    +
    +  /*
    +   * Register a serializer from Python, should be called during initialization
    +   */
    +  def register(ser: PythonTransformFunctionSerializer): Unit = {
    +    serializer = ser
    +  }
    +
    +  def serialize(func: PythonTransformFunction): Array[Byte] = {
    +    assert(serializer != null, "Serializer has not been registered!")
    +    // get the id of PythonTransformFunction in py4j
    +    val h = Proxy.getInvocationHandler(func.asInstanceOf[Proxy])
    +    val f = h.getClass().getDeclaredField("id")
    +    f.setAccessible(true)
    +    val id = f.get(h).asInstanceOf[String]
    +    serializer.dumps(id)
    +  }
    +
    +  def deserialize(bytes: Array[Byte]): PythonTransformFunction = {
    +    assert(serializer != null, "Serializer has not been registered!")
    +    serializer.loads(bytes)
    +  }
    +}
    +
    +/**
    + * Helper functions
    --- End diff --
    
    Maybe add a note here saying that these functions are called from Python via Py4J.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57758335
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21242/consoleFull) for   PR 2538 at commit [`e108ec1`](https://github.com/apache/spark/commit/e108ec114eb1a14c6e2387761da8e55bee4b3c83).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57856325
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21271/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18077695
  
    --- Diff: python/pyspark/streaming/jtime.py ---
    @@ -0,0 +1,135 @@
    +#
    +# 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 pyspark.streaming.duration import Duration
    +
    +"""
    +The name of this file, time is not a good naming for python
    --- End diff --
    
    Another reason streaming/time.py is not good is that time is used in python/pyspark/streaming/tests.py
    
    ```
                # StreamingContext.awaitTermination is not used to wait because
                # if py4j server is called every 50 milliseconds, it gets an error.
                time.sleep(0.05)
                # Check if the output is the same length of expected output.
                if len(expected) == len(result):
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18192144
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -52,6 +52,11 @@ private[spark] class PythonRDD(
         accumulator: Accumulator[JList[Array[Byte]]])
       extends RDD[Array[Byte]](parent) {
     
    +  def copyTo(rdd: RDD[_]): PythonRDD = {
    --- End diff --
    
    Please add a doc comment on what does this method do. If it essentially clones a PythonRDD, it maybe worthwhile to call this `clone()`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18079928
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,400 @@
    +#
    +# 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
    +
    +from pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction, RDDFunction2
    +from pyspark.rdd import portable_hash
    +from pyspark.streaming.duration import Seconds
    +
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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 print_(self, label=None):
    +        """
    +        Since print is reserved name for python, we cannot define a "print" method function.
    +        This function prints serialized data in RDD in DStream because Scala and Java cannot
    +        deserialized pickled python object. Please use DStream.pyprint() to print results.
    +
    +        Call DStream.print() and this function will print byte array in the DStream
    +        """
    +        # a hack to call print function in DStream
    +        getattr(self._jdstream, "print")(label)
    --- End diff --
    
    I am not sure print_ is needed for PythonSparkStreaming.
    Please remove "(label)"
    ```
    getattr(self._jdstream, "print")(label)
    ```
    
    I put this input parameter for debugging purpose to make sure which print is called when I use many print operation.
    
    ```
    -------------------------------------------
    Time: 123456789
    -------------------------------------------
    called after reduce
    ("a", 1)
    ("b", 22)
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18372671
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,305 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    --- End diff --
    
    Should `duration` be `batchDuration`, for parity with the Scala version?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56994578
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20859/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18078104
  
    --- Diff: examples/src/main/python/streaming/wordcount.py ---
    @@ -0,0 +1,21 @@
    +import sys
    +
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import *
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 2:
    +        print >> sys.stderr, "Usage: wordcount <directory>"
    +        exit(-1)
    +
    +    ssc = StreamingContext(appName="PythonStreamingWordCount",
    --- End diff --
    
    Based on your changes, this line should be
    ```
    sc = SparkContext(appName="PythonStreamingWordCount")
    ssc = StreamingContext(sc, Seconds(1))
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57129237
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20962/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18133131
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -288,7 +293,7 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce
      * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python.
      * This is used by PySpark's shuffle operations.
      */
    -private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
    +private[spark] class PairwiseRDD(prev: RDD[Array[Byte]]) extends
    --- End diff --
    
    This change is not necessary anymore because PairwiseRDD is not called in PyhotnDStream. I think just 'private' is enough.
    
    ```
    -private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
    +private[spark] class PairwiseRDD(prev: RDD[Array[Byte]]) extends  # This is not needed any more.
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57203903
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20981/consoleFull) for   PR 2538 at commit [`c40c52d`](https://github.com/apache/spark/commit/c40c52df9fd8b6dc8fd44196a73d57bd97a43a06).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57044911
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20909/consoleFull) for   PR 2538 at commit [`3f0fb4b`](https://github.com/apache/spark/commit/3f0fb4b7e8265c9076077bc8290aeac3b9aeb18b).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58115555
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21347/consoleFull) for   PR 2538 at commit [`8380064`](https://github.com/apache/spark/commit/8380064dcabdc38e0c3977b5a8a4612d2b5ea727).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58733406
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21612/consoleFull) for   PR 2538 at commit [`3e2492b`](https://github.com/apache/spark/commit/3e2492b9b95e0cc0e3427265f71f069000cc43f7).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57646701
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/256/consoleFull) for   PR 2538 at commit [`54bd92b`](https://github.com/apache/spark/commit/54bd92b5800ce9165e53289c44603e6a89c5ed75).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `          println(s"Failed to load main class $childMainClass.")`
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58774536
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21660/consoleFull) for   PR 2538 at commit [`64561e4`](https://github.com/apache/spark/commit/64561e4e503eafb958f6769383ba3b37edbe5fa2).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57045714
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20913/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18202907
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    +        """
    +        Return the first RDD in the stream.
    +        """
    +        return self._take(1)[0]
    +
    +    def _take(self, n):
    +        """
    +        Return the first `n` RDDs in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        self.foreachRDD(take)
    +
    +        self._ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self._ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            result.append(r)
    +        self.foreachRDD(get_output)
    +        return result
    +
    +    def mapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a map function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        map_values_fn = lambda (k, v): (k, f(v))
    +        return self.map(map_values_fn, preservesPartitioning=True)
    +
    +    def flatMapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a flatMap function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        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. Applying glom() to an RDD coalesces all
    +        elements within each partition into an list.
    +        """
    +        def func(iterator):
    +            yield list(iterator)
    +        return self.mapPartitions(func)
    +
    +    def cache(self):
    +        """
    +        Persist 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):
    +        """
    +        Set this DStream's storage level to persist its values across operations
    +        after the first time it is computed. This can only be used to assign
    +        a new storage level if the DStream does not have a storage level set yet.
    +        """
    +        self.is_cached = True
    +        javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
    +        self._jdstream.persist(javaStorageLevel)
    +        return self
    +
    +    def checkpoint(self, interval):
    +        """
    +        Mark this DStream for checkpointing. It will be saved to a file inside the
    +        checkpoint directory set with L{SparkContext.setCheckpointDir()}
    +
    +        @param interval: time in seconds, after which 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 which contains group the values for each key in the
    +        DStream into a single sequence.
    +        Hash-partitions the resulting RDD with into numPartitions partitions in
    +        the DStream.
    +
    +        Note: If you are grouping in order to perform an aggregation (such as a
    +        sum or average) over each key, using reduceByKey will provide much
    +        better performance.
    +        """
    +        return self.transform(lambda rdd: rdd.groupByKey(numPartitions))
    +
    +    def countByValue(self):
    +        """
    +        Return new DStream which contains the count of each unique value in this
    +        DStreeam as a (value, count) pairs.
    +        """
    +        return self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None).count()
    +
    +    def saveAsTextFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a text file, using string representations of elements.
    +        """
    +
    +        def saveAsTextFile(time, rdd):
    +            """
    +            Closure to save element in RDD in DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsTextFile(path)
    +
    +        return self.foreachRDD(saveAsTextFile)
    +
    +    def saveAsPickleFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a SequenceFile of serialized objects. The serializer
    +        used is L{pyspark.serializers.PickleSerializer}, default batch size
    +        is 10.
    +        """
    +
    +        def saveAsPickleFile(time, rdd):
    +            """
    +            Closure to save element in RDD in the DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsPickleFile(path)
    +
    +        return self.foreachRDD(saveAsPickleFile)
    +
    +    def transform(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, lambda t, a: func(a), True)
    +
    +    def transformWithTime(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, func, False)
    +
    +    def transformWith(self, func, other, keepSerializer=False):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream and 'other' DStream.
    +        """
    +        jfunc = RDDFunction(self.ctx, lambda t, a, b: func(a, b), self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonTransformed2DStream(self._jdstream.dstream(),
    +                                                          other._jdstream.dstream(), jfunc)
    +        jrdd_serializer = self._jrdd_deserializer if keepSerializer else self.ctx.serializer
    +        return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer)
    +
    +    def repartitions(self, numPartitions):
    +        """
    +        Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
    +        returned DStream has exactly numPartitions partitions.
    +        """
    +        return self.transform(lambda rdd: rdd.repartition(numPartitions))
    +
    +    @property
    +    def _slideDuration(self):
    +        """
    +        Return the slideDuration in seconds of this DStream
    +        """
    +        return self._jdstream.dstream().slideDuration().milliseconds() / 1000.0
    +
    +    def union(self, other):
    +        """
    +        Return a new DStream by unifying data of another DStream with this DStream.
    +        @param other Another DStream having the same interval (i.e., slideDuration) as this DStream.
    +        """
    +        if self._slideDuration != other._slideDuration:
    +            raise ValueError("the two DStream should have same slide duration")
    +        return self.transformWith(lambda a, b: a.union(b), other, True)
    +
    +    def cogroup(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'cogroup' between RDDs of `this`
    +        DStream and `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.cogroup(b, numPartitions), other)
    +
    +    def join(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.join(b, numPartitions), other)
    +
    +    def leftOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.leftOuterJoin(b, numPartitions), other)
    +
    +    def rightOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.rightOuterJoin(b, numPartitions), other)
    +
    +    def fullOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.fullOuterJoin(b, numPartitions), other)
    +
    +    def _jtime(self, timestamp):
    +        """ Convert datetime or unix_timestamp into Time
    +        """
    +        if isinstance(timestamp, datetime):
    +            timestamp = time.mktime(timestamp.timetuple())
    +        return self.ctx._jvm.Time(long(timestamp * 1000))
    +
    +    def slice(self, begin, end):
    +        """
    +        Return all the RDDs between 'begin' to 'end' (both included)
    +
    +        `begin`, `end` could be datetime.datetime() or unix_timestamp
    +        """
    +        jrdds = self._jdstream.slice(self._jtime(begin), self._jtime(end))
    +        return [RDD(jrdd, self.ctx, self._jrdd_deserializer) for jrdd in jrdds]
    +
    +    def _check_window(self, window, slide):
    +        duration = self._jdstream.dstream().slideDuration().milliseconds()
    +        if int(window * 1000) % duration != 0:
    +            raise ValueError("windowDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +        if slide and int(slide * 1000) % duration != 0:
    +            raise ValueError("slideDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +
    +    def window(self, windowDuration, slideDuration=None):
    +        """
    +        Return a new DStream in which each RDD contains all the elements in seen in a
    +        sliding window of time over this DStream.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        """
    +        self._check_window(windowDuration, slideDuration)
    +        d = self._ssc._jduration(windowDuration)
    +        if slideDuration is None:
    +            return DStream(self._jdstream.window(d), self._ssc, self._jrdd_deserializer)
    +        s = self._ssc._jduration(slideDuration)
    +        return DStream(self._jdstream.window(d, s), self._ssc, self._jrdd_deserializer)
    +
    +    def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated by reducing all
    +        elements in a sliding window over this DStream.
    +
    +        if `invReduceFunc` is not None, the reduction is done incrementally
    +        using the old window's reduced value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +         This is more efficient than `invReduceFunc` is None.
    +
    +        @param reduceFunc associative reduce function
    +        @param invReduceFunc inverse reduce function of `reduceFunc`
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        """
    +        keyed = self.map(lambda x: (1, x))
    +        reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc,
    +                                             windowDuration, slideDuration, 1)
    +        return reduced.map(lambda (k, v): v)
    +
    +    def countByWindow(self, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated
    +        by counting the number of elements in a window over this DStream.
    +        windowDuration and slideDuration are as defined in the window() operation.
    +
    +        This is equivalent to window(windowDuration, slideDuration).count(),
    +        but will be more efficient if window is large.
    +        """
    +        return self.map(lambda x: 1).reduceByWindow(operator.add, operator.sub,
    +                                                    windowDuration, slideDuration)
    +
    +    def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream in which each RDD contains the count of distinct elements in
    +        RDDs in a sliding window over this DStream.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  number of partitions of each RDD in the new DStream.
    +        """
    +        keyed = self.map(lambda x: (x, 1))
    +        counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub,
    +                                             windowDuration, slideDuration, numPartitions)
    +        return counted.filter(lambda (k, v): v > 0).count()
    +
    +    def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream by applying `groupByKey` over a sliding window.
    +        Similar to `DStream.groupByKey()`, but applies it over a sliding window.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  Number of partitions of each RDD in the new DStream.
    +        """
    +        ls = self.mapValues(lambda x: [x])
    +        grouped = ls.reduceByKeyAndWindow(lambda a, b: a.extend(b) or a, lambda a, b: a[len(b):],
    +                                          windowDuration, slideDuration, numPartitions)
    +        return grouped.mapValues(ResultIterable)
    +
    +    def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None,
    +                             numPartitions=None, filterFunc=None):
    +        """
    +        Return a new DStream by applying incremental `reduceByKey` over a sliding window.
    +
    +        The reduced value of over a new window is calculated using the old window's reduce value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +
    +        `invFunc` can be None, then it will reduce all the RDDs in window, could be slower
    +        than having `invFunc`.
    +
    +        @param reduceFunc     associative reduce function
    +        @param invReduceFunc  inverse function of `reduceFunc`
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  number of partitions of each RDD in the new DStream.
    +        @param filterFunc     function to filter expired key-value pairs;
    +                              only pairs that satisfy the function are retained
    +                              set this to null if you do not want to filter
    +        """
    +        self._check_window(windowDuration, slideDuration)
    +        reduced = self.reduceByKey(func)
    +
    +        def reduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            r = a.union(b).reduceByKey(func, numPartitions) if a else b
    +            if filterFunc:
    +                r = r.filter(filterFunc)
    +            return r
    +
    +        def invReduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            joined = a.leftOuterJoin(b, numPartitions)
    +            return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1)
    +
    +        jreduceFunc = RDDFunction(self.ctx, reduceFunc, reduced._jrdd_deserializer)
    +        if invReduceFunc:
    +            jinvReduceFunc = RDDFunction(self.ctx, invReduceFunc, reduced._jrdd_deserializer)
    +        else:
    +            jinvReduceFunc = None
    +        if slideDuration is None:
    +            slideDuration = self._slideDuration
    +        dstream = self.ctx._jvm.PythonReducedWindowedDStream(reduced._jdstream.dstream(),
    +                                                             jreduceFunc, jinvReduceFunc,
    +                                                             self._ssc._jduration(windowDuration),
    +                                                             self._ssc._jduration(slideDuration))
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +    def updateStateByKey(self, updateFunc, numPartitions=None):
    +        """
    +        Return a new "state" DStream where the state for each key is updated by applying
    +        the given function on the previous state of the key and the new values of the key.
    +
    +        @param updateFunc State update function ([(k, vs, s)] -> [(k, s)]).
    +                          If `s` is None, then `k` will be eliminated.
    +        """
    +        def reduceFunc(t, a, b):
    +            if a is None:
    +                g = b.groupByKey(numPartitions).map(lambda (k, vs): (k, list(vs), None))
    +            else:
    +                g = a.cogroup(b, numPartitions)
    +                g = g.map(lambda (k, (va, vb)): (k, list(vb), list(va)[0] if len(va) else None))
    +            state = g.mapPartitions(lambda x: updateFunc(x))
    +            return state.filter(lambda (k, v): v is not None)
    +
    +        jreduceFunc = RDDFunction(self.ctx, reduceFunc,
    +                                  self.ctx.serializer, self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonStateDStream(self._jdstream.dstream(), jreduceFunc)
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +
    +class TransformedDStream(DStream):
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18235440
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,275 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  // callback in Python
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    PythonDStream.some(pfunc.call(time.milliseconds, List(PythonDStream.wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(PythonDStream.wrapRDD(rdd), PythonDStream.wrapRDD(rdd2)).asJava
    +    PythonDStream.some(pfunc.call(time.milliseconds, rdds))
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +
    +/**
    + * Base class for PythonDStream with some common methods
    + */
    +private[python]
    +abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * Helper functions
    + */
    +private[python] object PythonDStream {
    +
    +  // convert Option[RDD[_]] to JavaRDD, handle null gracefully
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  // convert JavaRDD to Option[RDD[Array[Byte]]] to , handle null gracefully
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pfunc: PythonRDDFunction){
    --- End diff --
    
    Need space between `)` and `{`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18132354
  
    --- Diff: examples/src/main/python/streaming/network_wordcount.py ---
    @@ -0,0 +1,20 @@
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 3:
    +        print >> sys.stderr, "Usage: wordcount <hostname> <port>"
    +        exit(-1)
    +    sc = SparkContext(appName="PythonStreamingNetworkWordCount")
    +    ssc = StreamingContext(sc, 1)
    +
    +    lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2]))
    +    counts = lines.flatMap(lambda line: line.split(" "))\
    +                  .map(lambda word: (word, 1))\
    +                  .reduceByKey(lambda a, b: a+b)
    +    counts.pyprint()
    --- End diff --
    
    counts.pyprint() should be counts.pprint()
    
    ```
        def pprint(self):
            """
            Print the first ten elements of each RDD generated in this DStream. This is an output
            operator, so this DStream will be registered as an output stream and there materialized.
            """
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18379709
  
    --- 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 --
    
    I added this function as public method at first. That's why davices left this function in DStream. As far as my understanding, saveAsPickleFile is saveAsObjectFile in Scala/Java. If so, it would be good to be a public method.
    
    Python saveAsPickleFile doc string
    ```
    def saveAsPickleFile(self, path, batchSize=10): 
             """ 
              Save this RDD as a SequenceFile of serialized objects. The serializer 
              used is L{pyspark.serializers.PickleSerializer}, default batch size 
              is 10. 
             """
    ```
    
    
    Scala
    ```
    def saveAsObjectFile(path: String): Unit
    Save this RDD as a SequenceFile of serialized objects.
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58273668
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/281/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18251982
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,301 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.io.{ObjectInputStream, ObjectOutputStream}
    +import java.lang.reflect.Proxy
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + * TODO: support checkpoint
    + */
    +private[python] class RDDFunction(@transient var pfunc: PythonRDDFunction)
    --- End diff --
    
    I still find the name very confusing, even after looking at this code twice. Its more intuitive to call is TransformFunction (that is, function for transform operation) that RDDFunction (makes me think function of RDD operation, which is wrong). Please rename `RDDFunction` to `TransformFunction` and `PythonRDDFunction` to `PythonTransformFunction`. 
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57020039
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20876/consoleFull) for   PR 2538 at commit [`c28f520`](https://github.com/apache/spark/commit/c28f520ec2e77c6a5f7139b5131182024eddd1be).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class IDF(val minDocFreq: Int) `
      * `  class DocumentFrequencyAggregator(val minDocFreq: Int) extends Serializable `
      * `class PStatsParam(AccumulatorParam):`
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class Duration(object):`
      * `class Time(object):`
      * `class RDDFunction(object):`
      * `class RDDFunction2(object):`
      * `trait PythonRDDFunction `
      * `trait PythonRDDFunction2 `
      * `class PythonReducedWindowedDStream(`
      * `class PythonStateDStream(`
      * `class PythonForeachDStream(`
      * `class PythonDataInputStream(`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57204096
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20981/consoleFull) for   PR 2538 at commit [`c40c52d`](https://github.com/apache/spark/commit/c40c52df9fd8b6dc8fd44196a73d57bd97a43a06).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) extends Serializable `
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class RDDFunction(object):`
      * `trait PythonRDDFunction `
      * `class RDDFunction(pfunc: PythonRDDFunction)`
      * `abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) `
      * `class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,`
      * `class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],`
      * `class PythonStateDStream(parent: DStream[Array[Byte]], preduceFunc: PythonRDDFunction)`
      * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58281139
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21413/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58739352
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21631/consoleFull) for   PR 2538 at commit [`64561e4`](https://github.com/apache/spark/commit/64561e4e503eafb958f6769383ba3b37edbe5fa2).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58687330
  
    @davies #2624 is merged. Are we waiting for py4j 9.0 release?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58282175
  
    @tdas It looks like the tests are stable enough, it had 5 successes in a row.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56978050
  
    @giwa I will try to use random number for it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57284254
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21029/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57203546
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20980/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200173
  
    --- Diff: bin/pyspark ---
    @@ -87,11 +87,7 @@ export PYSPARK_SUBMIT_ARGS
     if [[ -n "$SPARK_TESTING" ]]; then
       unset YARN_CONF_DIR
       unset HADOOP_CONF_DIR
    -  if [[ -n "$PYSPARK_DOC_TEST" ]]; then
    -    exec "$PYSPARK_PYTHON" -m doctest $1
    -  else
    -    exec "$PYSPARK_PYTHON" $1
    -  fi
    +  exec "$PYSPARK_PYTHON" $1
    --- End diff --
    
    If that is easy to do, then that is a better idea. Since this PR is already so big, lets change as little of the existing infrastructure as possible. Otherwise if existing pyspark breaks in some weird way, it will be hard to revert this commit, without reverting all of pysparkstreaming.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58264478
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/281/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58738863
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21627/consoleFull) for   PR 2538 at commit [`331ecce`](https://github.com/apache/spark/commit/331ecced6f61ad5183da5830f94f584bcc74e479).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18739441
  
    --- Diff: examples/src/main/python/streaming/stateful_network_wordcount.py ---
    @@ -0,0 +1,57 @@
    +#
    +# 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: stateful_network_wordcount.py <hostname> <port>
    +   <hostname> and <port> describe the TCP server that Spark Streaming
    +    would connect to receive data.
    +
    + To run this on your local machine, you need to first run a Netcat server
    +    `$ nc -lk 9999`
    + and then run the example
    +    `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \
    +        localhost 9999`
    +"""
    +
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 3:
    +        print >> sys.stderr, "Usage: stateful_network_wordcount.py <hostname> <port>"
    +        exit(-1)
    +    sc = SparkContext(appName="PythonStreamingNetworkWordCount")
    --- End diff --
    
    appName could be "PythonStreamingStatefulNetworkWordCount"


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57019919
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20876/consoleFull) for   PR 2538 at commit [`c28f520`](https://github.com/apache/spark/commit/c28f520ec2e77c6a5f7139b5131182024eddd1be).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57510975
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/224/consoleFull) for   PR 2538 at commit [`bd8a4c2`](https://github.com/apache/spark/commit/bd8a4c2516147f1e99cf1f6e721346c18db23a20).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193335
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in this DStream.
    --- End diff --
    
    This doc string is very misleading. This sounds as if it returns the total running count of all elements received in a stream. Please copy the Scala class doc string.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56903179
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20829/consoleFull) for   PR 2538 at commit [`7339be0`](https://github.com/apache/spark/commit/7339be0df7da9259b838affc33a1ed0cbee93156).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedRDD(DStream):`
      * `class Duration(object):`
      * `class Time(object):`
      * `class RDDFunction(object):`
      * `trait PythonRDDFunction `
      * `class PythonTransformedDStream (parent: DStream[_], parent2: DStream[_], func: PythonRDDFunction,`
      * `class PythonForeachDStream(`
      * `class PythonDataInputStream(`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18195081
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    --- End diff --
    
    What is the need for both Scala functions - RDDFunction and PythonRDDFunction.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56903180
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20829/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57729651
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21226/consoleFull) for   PR 2538 at commit [`37fe06f`](https://github.com/apache/spark/commit/37fe06fb743a1934d834d603e04b678110bc0fd5).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18373408
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,319 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import, JavaObject
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +
    +    Also, it will update the port number (0) with real port
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +            if not self.port:
    +                # update port with real port
    +                self.port = self.server_socket.getsockname()[1]
    +        except Exception as e:
    +            msg = 'An error occurred while trying to start the callback server: %s' % e
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    +        """
    +
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._jssc = jssc or self._initialize_context(self._sc, duration)
    +
    +    def _initialize_context(self, sc, duration):
    +        self._ensure_initialized()
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @classmethod
    +    def _ensure_initialized(cls):
    +        SparkContext._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        java_import(gw.jvm, "org.apache.spark.streaming.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.python.*")
    +
    +        # start callback server
    +        # getattr will fallback to JVM, so we cannot test by hasattr()
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            # use random port
    +            gw._start_callback_server(0)
    +            # gateway with real port
    +            gw._python_proxy_port = gw._callback_server.port
    +            # get the GatewayServer object in JVM by ID
    +            jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client)
    +            # update the port of CallbackClient with real port
    +            gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port)
    +
    +        # register serializer for TransformFunction
    +        # it happens before creating SparkContext when loading from checkpointing
    +        cls._transformerSerializer = TransformFunctionSerializer(
    +            SparkContext._active_spark_context, CloudPickleSerializer(), gw)
    +
    +    @classmethod
    +    def getOrCreate(cls, path, setupFunc):
    +        """
    +        Get the StreamingContext from checkpoint file at `path`, or setup
    +        it by `setupFunc`.
    +
    +        :param path: directory of checkpoint
    +        :param setupFunc: a function used to create StreamingContext and
    +                          setup DStreams.
    +        :return: a StreamingContext
    +        """
    +        if not os.path.exists(path) or not os.path.isdir(path) or not os.listdir(path):
    +            ssc = setupFunc()
    +            ssc.checkpoint(path)
    +            return ssc
    +
    +        cls._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        try:
    +            jssc = gw.jvm.JavaStreamingContext(path)
    +        except Exception:
    +            print >>sys.stderr, "failed to load StreamingContext from checkpoint"
    +            raise
    +
    +        jsc = jssc.sparkContext()
    +        conf = SparkConf(_jconf=jsc.getConf())
    +        sc = SparkContext(conf=conf, gateway=gw, jsc=jsc)
    +        # update ctx in serializer
    +        SparkContext._active_spark_context = sc
    +        cls._transformerSerializer.ctx = sc
    +        return StreamingContext(sc, None, jssc)
    +
    +    @property
    +    def sparkContext(self):
    +        """
    +        Return SparkContext which is associated with this StreamingContext.
    +        """
    +        return self._sc
    +
    +    def start(self):
    +        """
    +        Start the execution of the streams.
    +        """
    +        self._jssc.start()
    +
    +    def awaitTermination(self, timeout=None):
    +        """
    +        Wait for the execution to stop.
    +        @param timeout: time to wait in seconds
    +        """
    +        if timeout is None:
    +            self._jssc.awaitTermination()
    +        else:
    +            self._jssc.awaitTermination(int(timeout * 1000))
    +
    +    def stop(self, stopSparkContext=True, stopGraceFully=False):
    +        """
    +        Stop the execution of the streams, with option of ensuring all
    +        received data has been processed.
    +
    +        @param stopSparkContext: Stop the associated SparkContext or not
    +        @param stopGracefully: Stop gracefully by waiting for the processing
    +                              of all received data to be completed
    +        """
    +        self._jssc.stop(stopSparkContext, stopGraceFully)
    +        if stopSparkContext:
    +            self._sc.stop()
    +
    +    def remember(self, duration):
    +        """
    +        Set each DStreams in this context to remember RDDs it generated
    +        in the last given duration. DStreams remember RDDs only for a
    +        limited duration of time and releases them for garbage collection.
    +        This method allows the developer to specify how to long to remember
    +        the RDDs (if the developer wishes to query old data outside the
    +        DStream computation).
    +
    +        @param duration: Minimum duration (in seconds) that each DStream
    +                        should remember its RDDs
    +        """
    +        self._jssc.remember(self._jduration(duration))
    +
    +    def checkpoint(self, directory):
    +        """
    +        Sets the context to periodically checkpoint the DStream operations for master
    +        fault-tolerance. The graph will be checkpointed every batch interval.
    +
    +        @param directory: HDFS-compatible directory where the checkpoint data
    +                         will be reliably stored
    +        """
    +        self._jssc.checkpoint(directory)
    +
    +    def socketTextStream(self, hostname, port, storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2):
    +        """
    +        Create an input from TCP source hostname:port. Data is received using
    +        a TCP socket and receive byte is interpreted as UTF8 encoded ``\\n`` delimited
    +        lines.
    +
    +        @param hostname:      Hostname to connect to for receiving data
    +        @param port:          Port to connect to for receiving data
    +        @param storageLevel:  Storage level to use for storing the received objects
    +        """
    +        jlevel = self._sc._getJavaStorageLevel(storageLevel)
    +        return DStream(self._jssc.socketTextStream(hostname, port, jlevel), self,
    +                       UTF8Deserializer())
    +
    +    def textFileStream(self, directory):
    +        """
    +        Create an input stream that monitors a Hadoop-compatible file system
    +        for new files and reads them as text files. Files must be wrriten to the
    +        monitored directory by "moving" them from another location within the same
    +        file system. File names starting with . are ignored.
    +        """
    +        return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer())
    +
    +    def _check_serialzers(self, rdds):
    --- End diff --
    
    Minor typo: serialzers -> serializers.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57493111
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21126/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57120571
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20962/consoleFull) for   PR 2538 at commit [`b983f0f`](https://github.com/apache/spark/commit/b983f0fed06bcbd6e740fbf86af6eb8881e9f3fd).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18374241
  
    --- 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):
    +        """
    +        Save each RDD in this DStream as at binary file, the elements are
    +        serialized by pickle.
    +        """
    +        def saveAsPickleFile(t, rdd):
    +            path = rddToFileName(prefix, suffix, t)
    +            try:
    +                rdd.saveAsPickleFile(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(saveAsPickleFile)
    +
    +    def transform(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +
    +        `func` can have one argument of `rdd`, or have two arguments of
    +        (`time`, `rdd`)
    +        """
    +        if func.func_code.co_argcount == 1:
    +            oldfunc = func
    +            func = lambda t, rdd: oldfunc(rdd)
    +        assert func.func_code.co_argcount == 2, "func should take one or two arguments"
    +        return TransformedDStream(self, func)
    +
    +    def transformWith(self, func, other, keepSerializer=False):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream and 'other' DStream.
    +
    +        `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three
    +        arguments of (`time`, `rdd_a`, `rdd_b`)
    +        """
    +        if func.func_code.co_argcount == 2:
    +            oldfunc = func
    +            func = lambda t, a, b: oldfunc(a, b)
    +        assert func.func_code.co_argcount == 3, "func should take two or three arguments"
    +        jfunc = TransformFunction(self.ctx, func, self._jrdd_deserializer, other._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonTransformed2DStream(self._jdstream.dstream(),
    +                                                          other._jdstream.dstream(), jfunc)
    +        jrdd_serializer = self._jrdd_deserializer if keepSerializer else self.ctx.serializer
    +        return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer)
    +
    +    def repartition(self, numPartitions):
    +        """
    +        Return a new DStream with an increased or decreased level of parallelism.
    +        """
    +        return self.transform(lambda rdd: rdd.repartition(numPartitions))
    +
    +    @property
    +    def _slideDuration(self):
    +        """
    +        Return the slideDuration in seconds of this DStream
    +        """
    +        return self._jdstream.dstream().slideDuration().milliseconds() / 1000.0
    +
    +    def union(self, other):
    +        """
    +        Return a new DStream by unifying data of another DStream with this DStream.
    +
    +        @param other: Another DStream having the same interval (i.e., slideDuration)
    +                     as this DStream.
    +        """
    +        if self._slideDuration != other._slideDuration:
    +            raise ValueError("the two DStream should have same slide duration")
    +        return self.transformWith(lambda a, b: a.union(b), other, True)
    +
    +    def cogroup(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'cogroup' between RDDs of `this`
    +        DStream and `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +        return self.transformWith(lambda a, b: a.cogroup(b, numPartitions), other)
    +
    +    def join(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +        partitions.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +        return self.transformWith(lambda a, b: a.join(b, numPartitions), other)
    +
    +    def leftOuterJoin(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +        partitions.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +        return self.transformWith(lambda a, b: a.leftOuterJoin(b, numPartitions), other)
    +
    +    def rightOuterJoin(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +        partitions.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +        return self.transformWith(lambda a, b: a.rightOuterJoin(b, numPartitions), other)
    +
    +    def fullOuterJoin(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +        partitions.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +        return self.transformWith(lambda a, b: a.fullOuterJoin(b, numPartitions), other)
    +
    +    def _jtime(self, timestamp):
    +        """ Convert datetime or unix_timestamp into Time
    +        """
    +        if isinstance(timestamp, datetime):
    +            timestamp = time.mktime(timestamp.timetuple())
    +        return self.ctx._jvm.Time(long(timestamp * 1000))
    +
    +    def slice(self, begin, end):
    +        """
    +        Return all the RDDs between 'begin' to 'end' (both included)
    +
    +        `begin`, `end` could be datetime.datetime() or unix_timestamp
    +        """
    +        jrdds = self._jdstream.slice(self._jtime(begin), self._jtime(end))
    +        return [RDD(jrdd, self.ctx, self._jrdd_deserializer) for jrdd in jrdds]
    +
    +    def _validate_window_param(self, window, slide):
    +        duration = self._jdstream.dstream().slideDuration().milliseconds()
    +        if int(window * 1000) % duration != 0:
    +            raise ValueError("windowDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +        if slide and int(slide * 1000) % duration != 0:
    +            raise ValueError("slideDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +
    +    def window(self, windowDuration, slideDuration=None):
    +        """
    +        Return a new DStream in which each RDD contains all the elements in seen in a
    +        sliding window of time over this DStream.
    +
    +        @param windowDuration: width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration:  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        """
    +        self._validate_window_param(windowDuration, slideDuration)
    +        d = self._ssc._jduration(windowDuration)
    +        if slideDuration is None:
    +            return DStream(self._jdstream.window(d), self._ssc, self._jrdd_deserializer)
    +        s = self._ssc._jduration(slideDuration)
    +        return DStream(self._jdstream.window(d, s), self._ssc, self._jrdd_deserializer)
    +
    +    def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated by reducing all
    +        elements in a sliding window over this DStream.
    +
    +        if `invReduceFunc` is not None, the reduction is done incrementally
    +        using the old window's reduced value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +         This is more efficient than `invReduceFunc` is None.
    +
    +        @param reduceFunc:     associative reduce function
    +        @param invReduceFunc:  inverse reduce function of `reduceFunc`
    +        @param windowDuration: width of the window; must be a multiple of this DStream's
    +                               batching interval
    +        @param slideDuration:  sliding interval of the window (i.e., the interval after which
    +                               the new DStream will generate RDDs); must be a multiple of this
    +                               DStream's batching interval
    +        """
    +        keyed = self.map(lambda x: (1, x))
    +        reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc,
    +                                             windowDuration, slideDuration, 1)
    +        return reduced.map(lambda (k, v): v)
    +
    +    def countByWindow(self, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated
    +        by counting the number of elements in a window over this DStream.
    +        windowDuration and slideDuration are as defined in the window() operation.
    +
    +        This is equivalent to window(windowDuration, slideDuration).count(),
    +        but will be more efficient if window is large.
    +        """
    +        return self.map(lambda x: 1).reduceByWindow(operator.add, operator.sub,
    +                                                    windowDuration, slideDuration)
    +
    +    def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream in which each RDD contains the count of distinct elements in
    +        RDDs in a sliding window over this DStream.
    +
    +        @param windowDuration: width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration:  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions:  number of partitions of each RDD in the new DStream.
    +        """
    +        keyed = self.map(lambda x: (x, 1))
    +        counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub,
    +                                             windowDuration, slideDuration, numPartitions)
    +        return counted.filter(lambda (k, v): v > 0).count()
    +
    +    def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream by applying `groupByKey` over a sliding window.
    +        Similar to `DStream.groupByKey()`, but applies it over a sliding window.
    +
    +        @param windowDuration: width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration:  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions:  Number of partitions of each RDD in the new DStream.
    +        """
    +        ls = self.mapValues(lambda x: [x])
    +        grouped = ls.reduceByKeyAndWindow(lambda a, b: a.extend(b) or a, lambda a, b: a[len(b):],
    +                                          windowDuration, slideDuration, numPartitions)
    +        return grouped.mapValues(ResultIterable)
    +
    +    def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None,
    +                             numPartitions=None, filterFunc=None):
    +        """
    +        Return a new DStream by applying incremental `reduceByKey` over a sliding window.
    +
    +        The reduced value of over a new window is calculated using the old window's reduce value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +
    +        `invFunc` can be None, then it will reduce all the RDDs in window, could be slower
    +        than having `invFunc`.
    +
    +        @param reduceFunc:     associative reduce function
    +        @param invReduceFunc:  inverse function of `reduceFunc`
    +        @param windowDuration: width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration:  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions:  number of partitions of each RDD in the new DStream.
    +        @param filterFunc:     function to filter expired key-value pairs;
    +                              only pairs that satisfy the function are retained
    +                              set this to null if you do not want to filter
    +        """
    +        self._validate_window_param(windowDuration, slideDuration)
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +
    +        reduced = self.reduceByKey(func, numPartitions)
    +
    +        def reduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            r = a.union(b).reduceByKey(func, numPartitions) if a else b
    +            if filterFunc:
    +                r = r.filter(filterFunc)
    +            return r
    +
    +        def invReduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            joined = a.leftOuterJoin(b, numPartitions)
    +            return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1)
    +
    +        jreduceFunc = TransformFunction(self.ctx, reduceFunc, reduced._jrdd_deserializer)
    +        if invReduceFunc:
    +            jinvReduceFunc = TransformFunction(self.ctx, invReduceFunc, reduced._jrdd_deserializer)
    +        else:
    +            jinvReduceFunc = None
    +        if slideDuration is None:
    +            slideDuration = self._slideDuration
    +        dstream = self.ctx._jvm.PythonReducedWindowedDStream(reduced._jdstream.dstream(),
    +                                                             jreduceFunc, jinvReduceFunc,
    +                                                             self._ssc._jduration(windowDuration),
    +                                                             self._ssc._jduration(slideDuration))
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +    def updateStateByKey(self, updateFunc, numPartitions=None):
    +        """
    +        Return a new "state" DStream where the state for each key is updated by applying
    +        the given function on the previous state of the key and the new values of the key.
    +
    +        @param updateFunc: State update function ([(k, vs, s)] -> [(k, s)]).
    +                          If `s` is None, then `k` will be eliminated.
    +        """
    +        if numPartitions is None:
    +            numPartitions = self.ctx.defaultParallelism
    +
    +        def reduceFunc(t, a, b):
    +            if a is None:
    +                g = b.groupByKey(numPartitions).map(lambda (k, vs): (k, list(vs), None))
    +            else:
    +                g = a.cogroup(b, numPartitions)
    +                g = g.map(lambda (k, (va, vb)): (k, list(vb), list(va)[0] if len(va) else None))
    +            state = g.mapPartitions(lambda x: updateFunc(x))
    +            return state.filter(lambda (k, v): v is not None)
    +
    +        jreduceFunc = TransformFunction(self.ctx, reduceFunc,
    +                                        self.ctx.serializer, self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonStateDStream(self._jdstream.dstream(), jreduceFunc)
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +
    +class TransformedDStream(DStream):
    +    """
    +    TransformedDStream is an DStream generated by an Python function
    +    transforming each RDD of an DStream to another RDDs.
    +
    +    Multiple continuous transformations of DStream can be combined into
    +    one transformation.
    +    """
    +    def __init__(self, prev, func):
    +        ssc = prev._ssc
    --- End diff --
    
    Well, and the ctx line, too.  Just a minor nit / non-issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56926658
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20843/consoleFull)** after     a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193683
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    --- End diff --
    
    Unless absolutely necessary, lets not add such private, test-only methods to the dstream.py


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200194
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -114,6 +114,9 @@ def __ne__(self, other):
         def __repr__(self):
             return "<%s object>" % self.__class__.__name__
     
    +    def __hash__(self):
    +        return hash(str(self))
    --- End diff --
    
    Similar question: are the changes in this file necessary for streaming or was part of the refactoring?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57148324
  
    @davies 
    Python API for Streaming becomes much nicer. You are awesome! Tests have been failed so far. But it looks like other issue, I think. 
    Thanks to you, callback server is terminated properly even after error is occurred, as far as I know. 
    
    I got errors when I executed run-tests. I pasted log below. This error is not always happened. When I run only TestWindowFunctions by  ./bin/pyspark python/pyspark/streaming/tests.py TestWindowFunctions, I have not seen this error.
    
    My environment:
    Mac OS 10.10 
    Python 2.6.9
    Java 1.7.0_65
    Scala 2.10.3
    
    ```
    Traceback (most recent call last):
      File "/Users/ken/spark/davies/spark/python/pyspark/streaming/util.py", line 39, in call
        r = self.func(rdd, other, milliseconds)
      File "/Users/ken/spark/davies/spark/python/pyspark/streaming/dstream.py", line 145, in <lambda>
        jfunc = RDDFunction(self.ctx, lambda a, _, t: func(a, t), self._jrdd_deserializer)
      File "/Users/ken/spark/davies/spark/python/pyspark/streaming/dstream.py", line 199, in get_output
        r = rdd.collect()
      File "/Users/ken/spark/davies/spark/python/pyspark/rdd.py", line 656, in collect
        bytesInJava = self._jrdd.collect().iterator()
      File "/Users/ken/spark/davies/spark/python/lib/py4j-0.8.2.1-src.zip/py4j/java_gateway.py", line 538, in __call__
        self.target_id, self.name)
      File "/Users/ken/spark/davies/spark/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py", line 300, in get_return_value
        format(target_id, '.', name), value)
    Py4JJavaError: An error occurred while calling o7478.collect.
    : org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 34.0 failed 1 times, most recent failure: Lost task 0.0 in stage 34.0 (TID 190, localhost): java.io.IOException: PARSING_ERROR(2)
            org.xerial.snappy.SnappyNative.throw_error(SnappyNative.java:84)
            org.xerial.snappy.SnappyNative.uncompressedLength(Native Method)
            org.xerial.snappy.Snappy.uncompressedLength(Snappy.java:594)
            org.xerial.snappy.SnappyInputStream.readFully(SnappyInputStream.java:125)
            org.xerial.snappy.SnappyInputStream.readHeader(SnappyInputStream.java:88)
            org.xerial.snappy.SnappyInputStream.<init>(SnappyInputStream.java:58)
            org.apache.spark.io.SnappyCompressionCodec.compressedInputStream(CompressionCodec.scala:128)
            org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:216)
            org.apache.spark.broadcast.TorrentBroadcast.readObject(TorrentBroadcast.scala:170)
            sun.reflect.GeneratedMethodAccessor9.invoke(Unknown Source)
            sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            java.lang.reflect.Method.invoke(Method.java:606)
            java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1893)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.readObject(ObjectInputStream.java:370)
            java.util.ArrayList.readObject(ArrayList.java:771)
            sun.reflect.GeneratedMethodAccessor6.invoke(Unknown Source)
            sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            java.lang.reflect.Method.invoke(Method.java:606)
            java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1893)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.readObject(ObjectInputStream.java:370)
            scala.collection.immutable.$colon$colon.readObject(List.scala:362)
            sun.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)
            sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            java.lang.reflect.Method.invoke(Method.java:606)
            java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1893)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990)
            java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915)
            java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798)
            java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350)
            java.io.ObjectInputStream.readObject(ObjectInputStream.java:370)
            org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:62)
            org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:87)
            org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:60)
            org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
            org.apache.spark.scheduler.Task.run(Task.scala:54)
            org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:182)
            java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
            java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
            java.lang.Thread.run(Thread.java:745)
    Driver stacktrace:
         at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1190)
         at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1179)
         at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1178)
         at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
         at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
         at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1178)
         at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:693)
         at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:693)
         at scala.Option.foreach(Option.scala:236)
         at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:693)
         at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1396)
         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498)
         at akka.actor.ActorCell.invoke(ActorCell.scala:456)
         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237)
         at akka.dispatch.Mailbox.run(Mailbox.scala:219)
         at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386)
         at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
         at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
         at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
         at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
    
    ======================================================================
    FAIL: test_count_by_window (__main__.TestWindowFunctions)
    ----------------------------------------------------------------------
    Traceback (most recent call last):
      File "pyspark/streaming/tests.py", line 284, in test_count_by_window
        self._test_func(input, func, expected)
      File "pyspark/streaming/tests.py", line 78, in _test_func
        self.assertEqual(expected, result)
    AssertionError: [[1], [3], [6], [9], [12], [15], [11], [6]] != [[1], [12], [], [], [], [], [], []]
    
    ======================================================================
    FAIL: test_count_by_window_large (__main__.TestWindowFunctions)
    ----------------------------------------------------------------------
    Traceback (most recent call last):
      File "pyspark/streaming/tests.py", line 293, in test_count_by_window_large
        self._test_func(input, func, expected)
      File "pyspark/streaming/tests.py", line 78, in _test_func
        self.assertEqual(expected, result)
    AssertionError: [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]] != [[1]]
    
    ======================================================================
    FAIL: test_group_by_key_and_window (__main__.TestWindowFunctions)
    ----------------------------------------------------------------------
    Traceback (most recent call last):
      File "pyspark/streaming/tests.py", line 303, in test_group_by_key_and_window
        self._test_func(input, func, expected)
      File "pyspark/streaming/tests.py", line 78, in _test_func
        self.assertEqual(expected, result)
    AssertionError: [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])], [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]] != [[('a', [0])], [('a', [2, 3, 4])], [], [], [], [], []]
    
    ----------------------------------------------------------------------
    Ran 24 tests in 148.729s
    
    FAILED (failures=3)
    
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57020408
  
    @giwa @JoshRosen It support window functions and updateStateByKey now, should be function complete as Java API, I'd move on to add more docs and tests.
    
    We did not do the calculation of time in Python, so Time and Duration are not necessary. I would like to use datetime or seconds (float) instead of Time, and seconds (int/float) instead of Duration in the API, then Python programmer can easily access them, then I will plan to remove Time and Duration in Python. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57047695
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20912/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57729657
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21226/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57103181
  
    @davies Could you add pyspark/streaming/util.py as a doc test?  It has a doc test.
    
    ```
     PYSPARK_DOC_TEST=1
     run_test "pyspark/broadcast.py"
     run_test "pyspark/accumulators.py"
     run_test "pyspark/serializers.py"
     run_test "pyspark/streaming/util.py"  # add this please
    ```
    
    python/pyspark/streaming/util.py
    ```
    +def rddToFileName(prefix, suffix, time):
    +    """
    +    Return string prefix-time(.suffix)
    +
    +    >>> rddToFileName("spark", None, 12345678910)
    +    'spark-12345678910'
    +    >>> rddToFileName("spark", "tmp", 12345678910)
    +    'spark-12345678910.tmp'
    +
    +    """
    +    if suffix is None:
    +        return prefix + "-" + str(time)
    +    else:
    +        return prefix + "-" + str(time) + "." + suffix
    ```
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18078083
  
    --- Diff: examples/src/main/python/streaming/network_wordcount.py ---
    @@ -0,0 +1,20 @@
    +import sys
    +
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import *
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 3:
    +        print >> sys.stderr, "Usage: wordcount <hostname> <port>"
    +        exit(-1)
    +    ssc = StreamingContext(appName="PythonStreamingNetworkWordCount",
    --- End diff --
    
    Based on your changes, this line should be
    ```
    sc = SparkContext(appName="PythonStreamingNetworkWordCount")
    ssc = StreamingContext(sc, Seconds(1))
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194319
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    --- End diff --
    
    `wrapRDD` and `some` does not seem to be using the `pfunc` and is therefore stateless. Shouldnt they be moved to an object? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58741207
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21627/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18373682
  
    --- 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
    --- End diff --
    
    We have a `context` method that returns the StreamingContext and this public `ctx` field that returns the underlying SparkContext.  This seems like it could be confusing, so maybe we should make the `ctx` field private.  On the other hand, RDD has a `ctx` field that returns the SparkContext, so this is consistent with that.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18074580
  
    --- Diff: python/pyspark/streaming/jtime.py ---
    @@ -0,0 +1,135 @@
    +#
    +# 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 pyspark.streaming.duration import Duration
    +
    +"""
    +The name of this file, time is not a good naming for python
    --- End diff --
    
    Will the hack break if we always import PySpark's random / time through a fully-qualified name, e.g. `pyspark.random` or `pyspark.time` and don't use relative imports internally?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57523695
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21151/consoleFull) for   PR 2538 at commit [`7a88f9f`](https://github.com/apache/spark/commit/7a88f9f1b054468b40e3134d7f4e0be8aacb03fa).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18373785
  
    --- 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):
    --- End diff --
    
    Do we need this method?  Can we just move the `reduce(operator.add)` to `count` and remove `_sum`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58736405
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21612/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57738844
  
    Seems like the unit test are failing with GC-overhead-too-high errors. @davies Please take a look. I can help out if you want. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18374009
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,319 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import, JavaObject
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +
    +    Also, it will update the port number (0) with real port
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +            if not self.port:
    +                # update port with real port
    +                self.port = self.server_socket.getsockname()[1]
    +        except Exception as e:
    +            msg = 'An error occurred while trying to start the callback server: %s' % e
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    +        """
    +
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._jssc = jssc or self._initialize_context(self._sc, duration)
    +
    +    def _initialize_context(self, sc, duration):
    +        self._ensure_initialized()
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @classmethod
    +    def _ensure_initialized(cls):
    +        SparkContext._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        java_import(gw.jvm, "org.apache.spark.streaming.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.python.*")
    +
    +        # start callback server
    +        # getattr will fallback to JVM, so we cannot test by hasattr()
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            # use random port
    +            gw._start_callback_server(0)
    +            # gateway with real port
    +            gw._python_proxy_port = gw._callback_server.port
    +            # get the GatewayServer object in JVM by ID
    +            jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client)
    +            # update the port of CallbackClient with real port
    +            gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port)
    +
    +        # register serializer for TransformFunction
    +        # it happens before creating SparkContext when loading from checkpointing
    +        cls._transformerSerializer = TransformFunctionSerializer(
    +            SparkContext._active_spark_context, CloudPickleSerializer(), gw)
    +
    +    @classmethod
    +    def getOrCreate(cls, path, setupFunc):
    +        """
    +        Get the StreamingContext from checkpoint file at `path`, or setup
    +        it by `setupFunc`.
    +
    +        :param path: directory of checkpoint
    +        :param setupFunc: a function used to create StreamingContext and
    +                          setup DStreams.
    +        :return: a StreamingContext
    +        """
    +        if not os.path.exists(path) or not os.path.isdir(path) or not os.listdir(path):
    +            ssc = setupFunc()
    +            ssc.checkpoint(path)
    +            return ssc
    +
    +        cls._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        try:
    +            jssc = gw.jvm.JavaStreamingContext(path)
    +        except Exception:
    +            print >>sys.stderr, "failed to load StreamingContext from checkpoint"
    +            raise
    +
    +        jsc = jssc.sparkContext()
    +        conf = SparkConf(_jconf=jsc.getConf())
    +        sc = SparkContext(conf=conf, gateway=gw, jsc=jsc)
    +        # update ctx in serializer
    +        SparkContext._active_spark_context = sc
    +        cls._transformerSerializer.ctx = sc
    +        return StreamingContext(sc, None, jssc)
    +
    +    @property
    +    def sparkContext(self):
    +        """
    +        Return SparkContext which is associated with this StreamingContext.
    +        """
    +        return self._sc
    +
    +    def start(self):
    +        """
    +        Start the execution of the streams.
    +        """
    +        self._jssc.start()
    +
    +    def awaitTermination(self, timeout=None):
    +        """
    +        Wait for the execution to stop.
    +        @param timeout: time to wait in seconds
    +        """
    +        if timeout is None:
    +            self._jssc.awaitTermination()
    +        else:
    +            self._jssc.awaitTermination(int(timeout * 1000))
    +
    +    def stop(self, stopSparkContext=True, stopGraceFully=False):
    +        """
    +        Stop the execution of the streams, with option of ensuring all
    +        received data has been processed.
    +
    +        @param stopSparkContext: Stop the associated SparkContext or not
    +        @param stopGracefully: Stop gracefully by waiting for the processing
    +                              of all received data to be completed
    +        """
    +        self._jssc.stop(stopSparkContext, stopGraceFully)
    +        if stopSparkContext:
    +            self._sc.stop()
    +
    +    def remember(self, duration):
    +        """
    +        Set each DStreams in this context to remember RDDs it generated
    +        in the last given duration. DStreams remember RDDs only for a
    +        limited duration of time and releases them for garbage collection.
    +        This method allows the developer to specify how to long to remember
    +        the RDDs (if the developer wishes to query old data outside the
    +        DStream computation).
    +
    +        @param duration: Minimum duration (in seconds) that each DStream
    +                        should remember its RDDs
    +        """
    +        self._jssc.remember(self._jduration(duration))
    +
    +    def checkpoint(self, directory):
    +        """
    +        Sets the context to periodically checkpoint the DStream operations for master
    +        fault-tolerance. The graph will be checkpointed every batch interval.
    +
    +        @param directory: HDFS-compatible directory where the checkpoint data
    +                         will be reliably stored
    +        """
    +        self._jssc.checkpoint(directory)
    +
    +    def socketTextStream(self, hostname, port, storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2):
    +        """
    +        Create an input from TCP source hostname:port. Data is received using
    +        a TCP socket and receive byte is interpreted as UTF8 encoded ``\\n`` delimited
    +        lines.
    +
    +        @param hostname:      Hostname to connect to for receiving data
    +        @param port:          Port to connect to for receiving data
    +        @param storageLevel:  Storage level to use for storing the received objects
    +        """
    +        jlevel = self._sc._getJavaStorageLevel(storageLevel)
    +        return DStream(self._jssc.socketTextStream(hostname, port, jlevel), self,
    +                       UTF8Deserializer())
    +
    +    def textFileStream(self, directory):
    +        """
    +        Create an input stream that monitors a Hadoop-compatible file system
    +        for new files and reads them as text files. Files must be wrriten to the
    +        monitored directory by "moving" them from another location within the same
    +        file system. File names starting with . are ignored.
    +        """
    +        return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer())
    +
    +    def _check_serialzers(self, rdds):
    +        # make sure they have same serializer
    +        if len(set(rdd._jrdd_deserializer for rdd in rdds)) > 1:
    +            for i in range(len(rdds)):
    +                # reset them to sc.serializer
    +                rdds[i] = rdds[i].map(lambda x: x, preservesPartitioning=True)
    +
    +    def queueStream(self, rdds, oneAtATime=True, default=None):
    +        """
    +        Create an input stream from an queue of RDDs or list. In each batch,
    +        it will process either one or all of the RDDs returned by the queue.
    +
    +        NOTE: changes to the queue after the stream is created will not be recognized.
    +
    +        @param rdds:       Queue of RDDs
    +        @param oneAtATime: pick one rdd each time or pick all of them once.
    +        @param default:    The default rdd if no more in rdds
    +        """
    +        if default and not isinstance(default, RDD):
    +            default = self._sc.parallelize(default)
    +
    +        if not rdds and default:
    +            rdds = [rdds]
    +
    +        if rdds and not isinstance(rdds[0], RDD):
    +            rdds = [self._sc.parallelize(input) for input in rdds]
    +        self._check_serialzers(rdds)
    +
    +        jrdds = ListConverter().convert([r._jrdd for r in rdds],
    +                                        SparkContext._gateway._gateway_client)
    +        queue = self._jvm.PythonDStream.toRDDQueue(jrdds)
    +        if default:
    +            default = default._reserialize(rdds[0]._jrdd_deserializer)
    +            jdstream = self._jssc.queueStream(queue, oneAtATime, default._jrdd)
    +        else:
    +            jdstream = self._jssc.queueStream(queue, oneAtATime)
    +        return DStream(jdstream, self, rdds[0]._jrdd_deserializer)
    +
    +    def transform(self, dstreams, transformFunc):
    +        """
    +        Create a new DStream in which each RDD is generated by applying
    +        a function on RDDs of the DStreams. The order of the JavaRDDs in
    +        the transform function parameter will be the same as the order
    +        of corresponding DStreams in the list.
    +        """
    +        jdstreams = ListConverter().convert([d._jdstream for d in dstreams],
    +                                            SparkContext._gateway._gateway_client)
    +        # change the final serializer to sc.serializer
    +        func = TransformFunction(self._sc,
    +                                 lambda t, *rdds: transformFunc(rdds).map(lambda x: x),
    +                                 *[d._jrdd_deserializer for d in dstreams])
    +        jfunc = self._jvm.TransformFunction(func)
    +        jdstream = self._jssc.transform(jdstreams, jfunc)
    +        return DStream(jdstream, self, self._sc.serializer)
    +
    +    def union(self, *dstreams):
    +        """
    +        Create a unified DStream from multiple DStreams of the same
    +        type and same slide duration.
    --- End diff --
    
    In `DStream.union`, we check the input to verify that the streams have the same slide duration (https://github.com/apache/spark/pull/2538/files#diff-6c576c52abc0624ccb6a2f45828dc6a7R326).  Can we do the same sort of input validation here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57974391
  
    @giwa maybe,this should be one of the most hottest PR (according to number of commits) : -)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57648549
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21206/consoleFull) for   PR 2538 at commit [`be5e5ff`](https://github.com/apache/spark/commit/be5e5ffdc5d2606042f09adb8d0fff08ddc4b85d).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57079953
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20932/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58281147
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21413/Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57647587
  
    @tdas I had included #2624 in this PR to make tests stable.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57227515
  
    @JoshRosen @tdas I think this PR is ready for review now, please take a look.
    
    @giwa I also saw these errors sometimes when I run the all the tests, it's a bit flaky, maybe related to start and stop of StreamingContext, I will investigate it later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58722649
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21604/consoleFull) for   PR 2538 at commit [`3e2492b`](https://github.com/apache/spark/commit/3e2492b9b95e0cc0e3427265f71f069000cc43f7).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18234481
  
    --- Diff: examples/src/main/python/streaming/hdfs_wordcount.py ---
    @@ -0,0 +1,21 @@
    +import sys
    +
    +from pyspark import SparkContext
    +from pyspark.streaming import StreamingContext
    +
    +if __name__ == "__main__":
    --- End diff --
    
    Similar to network_wordcount.py, can you add comments on how to run this example. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58774404
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18557210
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,319 @@
    +#
    +# 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.
    +#
    +import os
    +import sys
    +
    +from py4j.java_collections import ListConverter
    +from py4j.java_gateway import java_import, JavaObject
    +
    +from pyspark import RDD, SparkConf
    +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +
    +    The thread of callback server has daemon=False, it will block the driver
    +    from exiting if it's not shutdown. The following code replace `start()`
    +    of CallbackServer with a new version, which set daemon=True for this
    +    thread.
    +
    +    Also, it will update the port number (0) with real port
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +            if not self.port:
    +                # update port with real port
    +                self.port = self.server_socket.getsockname()[1]
    +        except Exception as e:
    +            msg = 'An error occurred while trying to start the callback server: %s' % e
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext
    +    represents the connection to a Spark cluster, and can be used to create
    +    L{DStream} various input sources. It can be from an existing L{SparkContext}.
    +    After creating and transforming DStreams, the streaming computation can
    +    be started and stopped using `context.start()` and `context.stop()`,
    +    respectively. `context.awaitTransformation()` allows the current thread
    +    to wait for the termination of the context by `stop()` or by an exception.
    +    """
    +    _transformerSerializer = None
    +
    +    def __init__(self, sparkContext, duration=None, jssc=None):
    +        """
    +        Create a new StreamingContext.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: number of seconds.
    +        """
    +
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._jssc = jssc or self._initialize_context(self._sc, duration)
    +
    +    def _initialize_context(self, sc, duration):
    +        self._ensure_initialized()
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @classmethod
    +    def _ensure_initialized(cls):
    +        SparkContext._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        java_import(gw.jvm, "org.apache.spark.streaming.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(gw.jvm, "org.apache.spark.streaming.api.python.*")
    +
    +        # start callback server
    +        # getattr will fallback to JVM, so we cannot test by hasattr()
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            # use random port
    +            gw._start_callback_server(0)
    +            # gateway with real port
    +            gw._python_proxy_port = gw._callback_server.port
    +            # get the GatewayServer object in JVM by ID
    +            jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client)
    +            # update the port of CallbackClient with real port
    +            gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port)
    +
    +        # register serializer for TransformFunction
    +        # it happens before creating SparkContext when loading from checkpointing
    +        cls._transformerSerializer = TransformFunctionSerializer(
    +            SparkContext._active_spark_context, CloudPickleSerializer(), gw)
    +
    +    @classmethod
    +    def getOrCreate(cls, path, setupFunc):
    +        """
    +        Get the StreamingContext from checkpoint file at `path`, or setup
    +        it by `setupFunc`.
    +
    +        :param path: directory of checkpoint
    +        :param setupFunc: a function used to create StreamingContext and
    +                          setup DStreams.
    +        :return: a StreamingContext
    +        """
    +        if not os.path.exists(path) or not os.path.isdir(path) or not os.listdir(path):
    +            ssc = setupFunc()
    +            ssc.checkpoint(path)
    +            return ssc
    +
    +        cls._ensure_initialized()
    +        gw = SparkContext._gateway
    +
    +        try:
    +            jssc = gw.jvm.JavaStreamingContext(path)
    +        except Exception:
    +            print >>sys.stderr, "failed to load StreamingContext from checkpoint"
    +            raise
    +
    +        jsc = jssc.sparkContext()
    +        conf = SparkConf(_jconf=jsc.getConf())
    +        sc = SparkContext(conf=conf, gateway=gw, jsc=jsc)
    +        # update ctx in serializer
    +        SparkContext._active_spark_context = sc
    +        cls._transformerSerializer.ctx = sc
    +        return StreamingContext(sc, None, jssc)
    +
    +    @property
    +    def sparkContext(self):
    +        """
    +        Return SparkContext which is associated with this StreamingContext.
    +        """
    +        return self._sc
    +
    +    def start(self):
    +        """
    +        Start the execution of the streams.
    +        """
    +        self._jssc.start()
    +
    +    def awaitTermination(self, timeout=None):
    +        """
    +        Wait for the execution to stop.
    +        @param timeout: time to wait in seconds
    +        """
    +        if timeout is None:
    +            self._jssc.awaitTermination()
    +        else:
    +            self._jssc.awaitTermination(int(timeout * 1000))
    +
    +    def stop(self, stopSparkContext=True, stopGraceFully=False):
    +        """
    +        Stop the execution of the streams, with option of ensuring all
    +        received data has been processed.
    +
    +        @param stopSparkContext: Stop the associated SparkContext or not
    +        @param stopGracefully: Stop gracefully by waiting for the processing
    +                              of all received data to be completed
    +        """
    +        self._jssc.stop(stopSparkContext, stopGraceFully)
    +        if stopSparkContext:
    +            self._sc.stop()
    +
    +    def remember(self, duration):
    +        """
    +        Set each DStreams in this context to remember RDDs it generated
    +        in the last given duration. DStreams remember RDDs only for a
    +        limited duration of time and releases them for garbage collection.
    +        This method allows the developer to specify how to long to remember
    +        the RDDs (if the developer wishes to query old data outside the
    +        DStream computation).
    +
    +        @param duration: Minimum duration (in seconds) that each DStream
    +                        should remember its RDDs
    +        """
    +        self._jssc.remember(self._jduration(duration))
    +
    +    def checkpoint(self, directory):
    +        """
    +        Sets the context to periodically checkpoint the DStream operations for master
    +        fault-tolerance. The graph will be checkpointed every batch interval.
    +
    +        @param directory: HDFS-compatible directory where the checkpoint data
    +                         will be reliably stored
    +        """
    +        self._jssc.checkpoint(directory)
    +
    +    def socketTextStream(self, hostname, port, storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2):
    +        """
    +        Create an input from TCP source hostname:port. Data is received using
    +        a TCP socket and receive byte is interpreted as UTF8 encoded ``\\n`` delimited
    +        lines.
    +
    +        @param hostname:      Hostname to connect to for receiving data
    +        @param port:          Port to connect to for receiving data
    +        @param storageLevel:  Storage level to use for storing the received objects
    +        """
    +        jlevel = self._sc._getJavaStorageLevel(storageLevel)
    +        return DStream(self._jssc.socketTextStream(hostname, port, jlevel), self,
    +                       UTF8Deserializer())
    +
    +    def textFileStream(self, directory):
    +        """
    +        Create an input stream that monitors a Hadoop-compatible file system
    +        for new files and reads them as text files. Files must be wrriten to the
    +        monitored directory by "moving" them from another location within the same
    +        file system. File names starting with . are ignored.
    +        """
    +        return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer())
    +
    +    def _check_serialzers(self, rdds):
    +        # make sure they have same serializer
    +        if len(set(rdd._jrdd_deserializer for rdd in rdds)) > 1:
    +            for i in range(len(rdds)):
    +                # reset them to sc.serializer
    +                rdds[i] = rdds[i].map(lambda x: x, preservesPartitioning=True)
    +
    +    def queueStream(self, rdds, oneAtATime=True, default=None):
    +        """
    +        Create an input stream from an queue of RDDs or list. In each batch,
    +        it will process either one or all of the RDDs returned by the queue.
    +
    +        NOTE: changes to the queue after the stream is created will not be recognized.
    +
    +        @param rdds:       Queue of RDDs
    +        @param oneAtATime: pick one rdd each time or pick all of them once.
    +        @param default:    The default rdd if no more in rdds
    +        """
    +        if default and not isinstance(default, RDD):
    +            default = self._sc.parallelize(default)
    +
    +        if not rdds and default:
    +            rdds = [rdds]
    +
    +        if rdds and not isinstance(rdds[0], RDD):
    +            rdds = [self._sc.parallelize(input) for input in rdds]
    +        self._check_serialzers(rdds)
    +
    +        jrdds = ListConverter().convert([r._jrdd for r in rdds],
    +                                        SparkContext._gateway._gateway_client)
    +        queue = self._jvm.PythonDStream.toRDDQueue(jrdds)
    +        if default:
    +            default = default._reserialize(rdds[0]._jrdd_deserializer)
    +            jdstream = self._jssc.queueStream(queue, oneAtATime, default._jrdd)
    +        else:
    +            jdstream = self._jssc.queueStream(queue, oneAtATime)
    +        return DStream(jdstream, self, rdds[0]._jrdd_deserializer)
    +
    +    def transform(self, dstreams, transformFunc):
    +        """
    +        Create a new DStream in which each RDD is generated by applying
    +        a function on RDDs of the DStreams. The order of the JavaRDDs in
    +        the transform function parameter will be the same as the order
    +        of corresponding DStreams in the list.
    +        """
    +        jdstreams = ListConverter().convert([d._jdstream for d in dstreams],
    +                                            SparkContext._gateway._gateway_client)
    +        # change the final serializer to sc.serializer
    +        func = TransformFunction(self._sc,
    +                                 lambda t, *rdds: transformFunc(rdds).map(lambda x: x),
    +                                 *[d._jrdd_deserializer for d in dstreams])
    +        jfunc = self._jvm.TransformFunction(func)
    +        jdstream = self._jssc.transform(jdstreams, jfunc)
    +        return DStream(jdstream, self, self._sc.serializer)
    +
    +    def union(self, *dstreams):
    +        """
    +        Create a unified DStream from multiple DStreams of the same
    +        type and same slide duration.
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58734420
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/346/consoleFull)**     for PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57279780
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21030/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18318446
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,304 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.io.{ObjectInputStream, ObjectOutputStream}
    +import java.lang.reflect.Proxy
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +private[python] trait PythonTransformFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonTransformFunction
    + * TODO: support checkpoint
    + */
    +private[python] class TransformFunction(@transient var pfunc: PythonTransformFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    Option(pfunc.call(time.milliseconds, List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava))
    +      .map(_.rdd)
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull, rdd2.map(JavaRDD.fromRDD(_)).orNull).asJava
    +    Option(pfunc.call(time.milliseconds, rdds)).map(_.rdd)
    +  }
    +
    +  // for function.Function2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +
    +  private def writeObject(out: ObjectOutputStream): Unit = {
    +    assert(PythonDStream.serializer != null, "Serializer has not been registered!")
    +    val bytes = PythonDStream.serializer.serialize(pfunc)
    +    out.writeInt(bytes.length)
    +    out.write(bytes)
    +  }
    +
    +  private def readObject(in: ObjectInputStream): Unit = {
    +    assert(PythonDStream.serializer != null, "Serializer has not been registered!")
    +    val length = in.readInt()
    +    val bytes = new Array[Byte](length)
    +    in.readFully(bytes)
    +    pfunc = PythonDStream.serializer.deserialize(bytes)
    +  }
    +}
    +
    +/**
    + * Interface for Python Serializer to serialize PythonTransformFunction
    + */
    +private[python] trait PythonTransformFunctionSerializer {
    +  def dumps(id: String): Array[Byte]  //
    --- End diff --
    
    Extra `//`
    nit: move this trait to be near `PythonTransformFunction`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57519148
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21141/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58774569
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/353/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57720268
  
    We should probably add Python tabs to the [Streaming Programing Guide](https://spark.apache.org/docs/latest/streaming-programming-guide.html).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57383500
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21066/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193747
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    +        """
    +        Return the first RDD in the stream.
    +        """
    +        return self._take(1)[0]
    +
    +    def _take(self, n):
    +        """
    +        Return the first `n` RDDs in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        self.foreachRDD(take)
    +
    +        self._ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self._ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            result.append(r)
    +        self.foreachRDD(get_output)
    +        return result
    +
    +    def mapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a map function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        map_values_fn = lambda (k, v): (k, f(v))
    +        return self.map(map_values_fn, preservesPartitioning=True)
    +
    +    def flatMapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a flatMap function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        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. Applying glom() to an RDD coalesces all
    +        elements within each partition into an list.
    +        """
    +        def func(iterator):
    +            yield list(iterator)
    +        return self.mapPartitions(func)
    +
    +    def cache(self):
    +        """
    +        Persist 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):
    +        """
    +        Set this DStream's storage level to persist its values across operations
    --- End diff --
    
    Wrong doc string, copy scala doc string.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58775690
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21660/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58775688
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21660/consoleFull) for   PR 2538 at commit [`64561e4`](https://github.com/apache/spark/commit/64561e4e503eafb958f6769383ba3b37edbe5fa2).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57427248
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21097/consoleFull) for   PR 2538 at commit [`6f0da2f`](https://github.com/apache/spark/commit/6f0da2fa486c2a580045a2e9e3133b6617875363).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class RDDFunction(object):`
      * `class RDDFunctionSerializer(object):`
      * `abstract class PythonDStream(parent: DStream[_], @transient pfunc: PythonRDDFunction)`
      * `class PythonTransformedDStream (parent: DStream[_], @transient pfunc: PythonRDDFunction,`
      * `class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],`
      * `class PythonStateDStream(parent: DStream[Array[Byte]], @transient reduceFunc: PythonRDDFunction)`
      * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58740718
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21631/Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57286037
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21031/consoleFull) for   PR 2538 at commit [`b98d63f`](https://github.com/apache/spark/commit/b98d63fbde10f20a42e1e6e0f34f45736b802772).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class RDDFunction(object):`
      * `abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)`
      * `class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,`
      * `class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],`
      * `class PythonStateDStream(parent: DStream[Array[Byte]], reduceFunc: PythonRDDFunction)`
      * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18192108
  
    --- Diff: bin/pyspark ---
    @@ -87,11 +87,7 @@ export PYSPARK_SUBMIT_ARGS
     if [[ -n "$SPARK_TESTING" ]]; then
       unset YARN_CONF_DIR
       unset HADOOP_CONF_DIR
    -  if [[ -n "$PYSPARK_DOC_TEST" ]]; then
    -    exec "$PYSPARK_PYTHON" -m doctest $1
    -  else
    -    exec "$PYSPARK_PYTHON" $1
    -  fi
    +  exec "$PYSPARK_PYTHON" $1
    --- End diff --
    
    Why is this change necessary?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57237943
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/196/consoleFull) for   PR 2538 at commit [`19797f9`](https://github.com/apache/spark/commit/19797f9fc9b062ee30746c184ad432192ca5e19a).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193135
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,243 @@
    +#
    +# 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 ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD
    +from pyspark.serializers import UTF8Deserializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import RDDFunction
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    +    """
    +    # TODO: create a patch for Py4J
    +    import socket
    +    import py4j.java_gateway
    +    logger = py4j.java_gateway.logger
    +    from py4j.java_gateway import Py4JNetworkError
    +    from threading import Thread
    +
    +    def start(self):
    +        """Starts the CallbackServer. This method should be called by the
    +        client instead of run()."""
    +        self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
    +        self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR,
    +                                      1)
    +        try:
    +            self.server_socket.bind((self.address, self.port))
    +            # self.port = self.server_socket.getsockname()[1]
    +        except Exception:
    +            msg = 'An error occurred while trying to start the callback server'
    +            logger.exception(msg)
    +            raise Py4JNetworkError(msg)
    +
    +        # Maybe thread needs to be cleanup up?
    +        self.thread = Thread(target=self.run)
    +        self.thread.daemon = True
    +        self.thread.start()
    +
    +    py4j.java_gateway.CallbackServer.start = start
    +
    +
    +class StreamingContext(object):
    +    """
    +    Main entry point for Spark Streaming functionality. A StreamingContext represents the
    +    connection to a Spark cluster, and can be used to create L{DStream}s and
    +    broadcast variables on that cluster.
    +    """
    +
    +    def __init__(self, sparkContext, duration):
    +        """
    +        Create a new StreamingContext. At least the master and app name and duration
    +        should be set, either through the named parameters here or through C{conf}.
    +
    +        @param sparkContext: L{SparkContext} object.
    +        @param duration: seconds for SparkStreaming.
    +
    +        """
    +        self._sc = sparkContext
    +        self._jvm = self._sc._jvm
    +        self._start_callback_server()
    +        self._jssc = self._initialize_context(self._sc, duration)
    +
    +    def _start_callback_server(self):
    +        gw = self._sc._gateway
    +        # getattr will fallback to JVM
    +        if "_callback_server" not in gw.__dict__:
    +            _daemonize_callback_server()
    +            gw._start_callback_server(gw._python_proxy_port)
    +            gw._python_proxy_port = gw._callback_server.port  # update port with real port
    +
    +    def _initialize_context(self, sc, duration):
    +        java_import(self._jvm, "org.apache.spark.streaming.*")
    +        java_import(self._jvm, "org.apache.spark.streaming.api.java.*")
    +        java_import(self._jvm, "org.apache.spark.streaming.api.python.*")
    +        return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration))
    +
    +    def _jduration(self, seconds):
    +        """
    +        Create Duration object given number of seconds
    +        """
    +        return self._jvm.Duration(int(seconds * 1000))
    +
    +    @property
    +    def sparkContext(self):
    +        """
    +        Return SparkContext which is associated with this StreamingContext.
    +        """
    +        return self._sc
    +
    +    def start(self):
    +        """
    +        Start the execution of the streams.
    +        """
    +        self._jssc.start()
    +
    +    def awaitTermination(self, timeout=None):
    +        """
    +        Wait for the execution to stop.
    +        @param timeout: time to wait in seconds
    +        """
    +        if timeout is None:
    +            self._jssc.awaitTermination()
    +        else:
    +            self._jssc.awaitTermination(int(timeout * 1000))
    +
    +    def stop(self, stopSparkContext=True, stopGraceFully=False):
    +        """
    +        Stop the execution of the streams immediately (does not wait for all received data
    +        to be processed).
    +        """
    +        self._jssc.stop(stopSparkContext, stopGraceFully)
    +        if stopSparkContext:
    +            self._sc.stop()
    +
    +    def remember(self, duration):
    +        """
    +        Set each DStreams in this context to remember RDDs it generated
    +        in the last given duration. DStreams remember RDDs only for a
    +        limited duration of time and releases them for garbage collection.
    +        This method allows the developer to specify how to long to remember
    +        the RDDs ( if the developer wishes to query old data outside the
    --- End diff --
    
    extra space after (


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18201113
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pyfunc: PythonRDDFunction){
    +    val func = new RDDFunction(pyfunc)
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    +    val func = new RDDFunction(pyfunc)
    +    ssc.transform(jdsteams, func)
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    + * this can reduce the Python callbacks.
    + */
    +private[spark]
    +class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,
    +                                var reuse: Boolean = false)
    +  extends PythonDStream(parent) {
    +
    +  val func = new RDDFunction(pfunc)
    +  var lastResult: PythonRDD = _
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val rdd1 = parent.getOrCompute(validTime)
    +    if (rdd1.isEmpty) {
    +      return None
    +    }
    +    if (reuse && lastResult != null) {
    +      Some(lastResult.copyTo(rdd1.get))
    +    } else {
    +      val r = func(rdd1, validTime)
    +      if (reuse && r.isDefined && lastResult == null) {
    +        r.get match {
    +          case rdd: PythonRDD =>
    +            if (rdd.parent(0) == rdd1) {
    +              // only one PythonRDD
    +              lastResult = rdd
    +            } else {
    +              // may have multiple stages
    +              reuse = false
    +            }
    +        }
    +      }
    +      r
    +    }
    +  }
    +}
    +
    +/**
    + * Transformed from two DStreams in Python.
    + */
    +private[spark]
    +class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],
    +                                pfunc: PythonRDDFunction)
    +  extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  val func = new RDDFunction(pfunc)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  override def dependencies = List(parent, parent2)
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    func(parent.getOrCompute(validTime), parent2.getOrCompute(validTime), validTime)
    +  }
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +/**
    + * similar to StateDStream
    + */
    +private[spark]
    +class PythonStateDStream(parent: DStream[Array[Byte]], preduceFunc: PythonRDDFunction)
    +  extends PythonDStream(parent) {
    +
    +  val reduceFunc = new RDDFunction(preduceFunc)
    +
    +  super.persist(StorageLevel.MEMORY_ONLY)
    +  override val mustCheckpoint = true
    +
    +  override def compute(validTime: Time): Option[RDD[Array[Byte]]] = {
    +    val lastState = getOrCompute(validTime - slideDuration)
    +    val rdd = parent.getOrCompute(validTime)
    +    if (rdd.isDefined) {
    +      reduceFunc(lastState, rdd, validTime)
    +    } else {
    +      lastState
    +    }
    +  }
    +}
    +
    +/**
    + * similar to ReducedWindowedDStream
    + */
    +private[spark]
    +class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],
    --- End diff --
    
    Why `private[spark]` and not `private[python]`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57511019
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21141/consoleFull) for   PR 2538 at commit [`bd8a4c2`](https://github.com/apache/spark/commit/bd8a4c2516147f1e99cf1f6e721346c18db23a20).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18193852
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    +        """
    +        Return the first RDD in the stream.
    +        """
    +        return self._take(1)[0]
    +
    +    def _take(self, n):
    +        """
    +        Return the first `n` RDDs in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        self.foreachRDD(take)
    +
    +        self._ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self._ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            result.append(r)
    +        self.foreachRDD(get_output)
    +        return result
    +
    +    def mapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a map function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        map_values_fn = lambda (k, v): (k, f(v))
    +        return self.map(map_values_fn, preservesPartitioning=True)
    +
    +    def flatMapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a flatMap function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        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. Applying glom() to an RDD coalesces all
    +        elements within each partition into an list.
    +        """
    +        def func(iterator):
    +            yield list(iterator)
    +        return self.mapPartitions(func)
    +
    +    def cache(self):
    +        """
    +        Persist 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):
    +        """
    +        Set this DStream's storage level to persist its values across operations
    +        after the first time it is computed. This can only be used to assign
    +        a new storage level if the DStream does not have a storage level set yet.
    +        """
    +        self.is_cached = True
    +        javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
    +        self._jdstream.persist(javaStorageLevel)
    +        return self
    +
    +    def checkpoint(self, interval):
    +        """
    +        Mark this DStream for checkpointing. It will be saved to a file inside the
    +        checkpoint directory set with L{SparkContext.setCheckpointDir()}
    +
    +        @param interval: time in seconds, after which 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 which contains group the values for each key in the
    +        DStream into a single sequence.
    +        Hash-partitions the resulting RDD with into numPartitions partitions in
    +        the DStream.
    +
    +        Note: If you are grouping in order to perform an aggregation (such as a
    +        sum or average) over each key, using reduceByKey will provide much
    +        better performance.
    +        """
    +        return self.transform(lambda rdd: rdd.groupByKey(numPartitions))
    +
    +    def countByValue(self):
    +        """
    +        Return new DStream which contains the count of each unique value in this
    +        DStreeam as a (value, count) pairs.
    +        """
    +        return self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None).count()
    +
    +    def saveAsTextFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a text file, using string representations of elements.
    +        """
    +
    +        def saveAsTextFile(time, rdd):
    +            """
    +            Closure to save element in RDD in DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsTextFile(path)
    +
    +        return self.foreachRDD(saveAsTextFile)
    +
    +    def saveAsPickleFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a SequenceFile of serialized objects. The serializer
    +        used is L{pyspark.serializers.PickleSerializer}, default batch size
    +        is 10.
    +        """
    +
    +        def saveAsPickleFile(time, rdd):
    +            """
    +            Closure to save element in RDD in the DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsPickleFile(path)
    +
    +        return self.foreachRDD(saveAsPickleFile)
    +
    +    def transform(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, lambda t, a: func(a), True)
    +
    +    def transformWithTime(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, func, False)
    +
    +    def transformWith(self, func, other, keepSerializer=False):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream and 'other' DStream.
    +        """
    +        jfunc = RDDFunction(self.ctx, lambda t, a, b: func(a, b), self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonTransformed2DStream(self._jdstream.dstream(),
    +                                                          other._jdstream.dstream(), jfunc)
    +        jrdd_serializer = self._jrdd_deserializer if keepSerializer else self.ctx.serializer
    +        return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer)
    +
    +    def repartitions(self, numPartitions):
    --- End diff --
    
    Should be `repartition`, not `repartitions`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194355
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    +  def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]]
    +}
    +
    +/**
    + * Wrapper for PythonRDDFunction
    + */
    +private[python] class RDDFunction(pfunc: PythonRDDFunction)
    +  extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] with Serializable {
    +
    +  def wrapRDD(rdd: Option[RDD[_]]): JavaRDD[_] = {
    +    if (rdd.isDefined) {
    +      JavaRDD.fromRDD(rdd.get)
    +    } else {
    +      null
    +    }
    +  }
    +
    +  def some(jrdd: JavaRDD[Array[Byte]]): Option[RDD[Array[Byte]]] = {
    +    if (jrdd != null) {
    +      Some(jrdd.rdd)
    +    } else {
    +      None
    +    }
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd)).asJava))
    +  }
    +
    +  def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = {
    +    some(pfunc.call(time.milliseconds, List(wrapRDD(rdd), wrapRDD(rdd2)).asJava))
    +  }
    +
    +  // for JFunction2
    +  def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = {
    +    pfunc.call(time.milliseconds, rdds)
    +  }
    +}
    +
    +private[python]
    +abstract class PythonDStream(parent: DStream[_]) extends DStream[Array[Byte]] (parent.ssc) {
    +
    +  override def dependencies = List(parent)
    +
    +  override def slideDuration: Duration = parent.slideDuration
    +
    +  val asJavaDStream  = JavaDStream.fromDStream(this)
    +}
    +
    +private[spark] object PythonDStream {
    +
    +  // helper function for DStream.foreachRDD(),
    +  // cannot be `foreachRDD`, it will confusing py4j
    +  def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pyfunc: PythonRDDFunction){
    +    val func = new RDDFunction(pyfunc)
    +    jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time))
    +  }
    +
    +  // helper function for ssc.transform()
    +  def callTransform(ssc: JavaStreamingContext, jdsteams: JList[JavaDStream[_]],
    +                    pyfunc: PythonRDDFunction)
    +    :JavaDStream[Array[Byte]] = {
    +    val func = new RDDFunction(pyfunc)
    +    ssc.transform(jdsteams, func)
    +  }
    +
    +  // convert list of RDD into queue of RDDs, for ssc.queueStream()
    +  def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = {
    +    val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]]
    +    rdds.forall(queue.add(_))
    +    queue
    +  }
    +}
    +
    +/**
    + * Transformed DStream in Python.
    + *
    + * If the result RDD is PythonRDD, then it will cache it as an template for future use,
    --- End diff --
    
    What do you mean by result RDD? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57224244
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20995/consoleFull) for   PR 2538 at commit [`6ebceca`](https://github.com/apache/spark/commit/6ebceca528dbd94dc23eba4412715e661ff6527e).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57535823
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21151/consoleFull) for   PR 2538 at commit [`7a88f9f`](https://github.com/apache/spark/commit/7a88f9f1b054468b40e3134d7f4e0be8aacb03fa).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57281449
  
    @davies That's great!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18192924
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -256,3 +256,8 @@ def _start_update_server():
         thread.daemon = True
         thread.start()
         return server
    +
    +
    +if __name__ == "__main__":
    --- End diff --
    
    Is this addition necessary?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58088738
  
    @davies @nchammas @shaneknapp Thank you for information. I hope we never suffer from this error.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56906120
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/20833/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57707329
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21219/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56919790
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20843/consoleFull) for   PR 2538 at commit [`9a57685`](https://github.com/apache/spark/commit/9a576859ca757b44b609bf9b823df5d6b376a98b).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18095580
  
    --- Diff: examples/src/main/python/streaming/wordcount.py ---
    @@ -0,0 +1,21 @@
    +import sys
    +
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import *
    +
    +if __name__ == "__main__":
    +    if len(sys.argv) != 2:
    +        print >> sys.stderr, "Usage: wordcount <directory>"
    +        exit(-1)
    +
    +    ssc = StreamingContext(appName="PythonStreamingWordCount",
    --- End diff --
    
    fixed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58273827
  
    Jenkins, test this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18200181
  
    --- Diff: python/pyspark/accumulators.py ---
    @@ -256,3 +256,8 @@ def _start_update_server():
         thread.daemon = True
         thread.start()
         return server
    +
    +
    +if __name__ == "__main__":
    --- End diff --
    
    Same logic as I mentioned earlier. If it is easy undo the refactoring and be able to run tests, lets try to do that. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18378532
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.io.{ObjectInputStream, ObjectOutputStream}
    +import java.lang.reflect.Proxy
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.language.existentials
    +
    +import py4j.GatewayServer
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    --- End diff --
    
    It looks like this now only takes two arguments.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57646825
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21203/consoleFull) for   PR 2538 at commit [`c7bbbce`](https://github.com/apache/spark/commit/c7bbbced7ba2d45e5fb2c1452920de11bd5138a8).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194704
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala ---
    @@ -0,0 +1,261 @@
    +/*
    + * 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.
    + */
    +
    +package org.apache.spark.streaming.api.python
    +
    +import java.util.{ArrayList => JArrayList, List => JList}
    +import scala.collection.JavaConversions._
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +
    +import org.apache.spark.api.java._
    +import org.apache.spark.api.python._
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.storage.StorageLevel
    +import org.apache.spark.streaming.{Interval, Duration, Time}
    +import org.apache.spark.streaming.dstream._
    +import org.apache.spark.streaming.api.java._
    +
    +
    +/**
    + * Interface for Python callback function with three arguments
    + */
    +trait PythonRDDFunction {
    --- End diff --
    
    Should be private to spark. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58740717
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21631/consoleFull) for   PR 2538 at commit [`64561e4`](https://github.com/apache/spark/commit/64561e4e503eafb958f6769383ba3b37edbe5fa2).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57077576
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/20932/consoleFull) for   PR 2538 at commit [`847f9b9`](https://github.com/apache/spark/commit/847f9b9faba9f9e6af20c9f5e72e68bc9eb52f4d).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58108774
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21347/consoleFull) for   PR 2538 at commit [`8380064`](https://github.com/apache/spark/commit/8380064dcabdc38e0c3977b5a8a4612d2b5ea727).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57656620
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21203/consoleFull) for   PR 2538 at commit [`c7bbbce`](https://github.com/apache/spark/commit/c7bbbced7ba2d45e5fb2c1452920de11bd5138a8).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18202884
  
    --- Diff: python/pyspark/streaming/context.py ---
    @@ -0,0 +1,243 @@
    +#
    +# 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 ListConverter
    +from py4j.java_gateway import java_import
    +
    +from pyspark import RDD
    +from pyspark.serializers import UTF8Deserializer
    +from pyspark.context import SparkContext
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.dstream import DStream
    +from pyspark.streaming.util import RDDFunction
    +
    +__all__ = ["StreamingContext"]
    +
    +
    +def _daemonize_callback_server():
    +    """
    +    Hack Py4J to daemonize callback server
    --- End diff --
    
    done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18148393
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala ---
    @@ -549,6 +549,10 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable {
      * JavaStreamingContext object contains a number of utility functions.
      */
     object JavaStreamingContext {
    +  implicit def fromStreamingContext(ssc: StreamingContext):
    +    JavaStreamingContext = new JavaStreamingContext(ssc)
    +
    +  implicit def toStreamingContext(jssc: JavaStreamingContext): StreamingContext = jssc.ssc
    --- End diff --
    
    This change is not needed anymore because these functions are not used.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57284243
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21029/consoleFull) for   PR 2538 at commit [`e00136b`](https://github.com/apache/spark/commit/e00136b3dfd330689d89e44006a49871b36a4825).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class RDDFunction(object):`
      * `abstract class PythonDStream(parent: DStream[_], pfunc: PythonRDDFunction)`
      * `class PythonTransformedDStream (parent: DStream[_], pfunc: PythonRDDFunction,`
      * `class PythonTransformed2DStream(parent: DStream[_], parent2: DStream[_],`
      * `class PythonStateDStream(parent: DStream[Array[Byte]], reduceFunc: PythonRDDFunction)`
      * `class PythonReducedWindowedDStream(parent: DStream[Array[Byte]],`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18194129
  
    --- Diff: python/pyspark/streaming/dstream.py ---
    @@ -0,0 +1,633 @@
    +#
    +# 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 pyspark import RDD
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.streaming.util import rddToFileName, RDDFunction
    +from pyspark.rdd import portable_hash
    +from pyspark.resultiterable import ResultIterable
    +
    +__all__ = ["DStream"]
    +
    +
    +class DStream(object):
    +    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 which contains the number of elements in 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):
    +        """
    +        Pass each value in the key-value pair DStream through flatMap function
    +        without changing the keys: this also retains the original RDD's partition.
    +        """
    +        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 by applying a function to each partition 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 by applying a function to each partition of this DStream,
    +        while tracking the index of the original partition.
    +        """
    +        return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning))
    +
    +    def reduce(self, func):
    +        """
    +        Return a new DStream by reduceing the elements of this RDD using the specified
    +        commutative and associative binary operator.
    +        """
    +        return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1])
    +
    +    def reduceByKey(self, func, numPartitions=None):
    +        """
    +        Merge the value for each key using an associative reduce function.
    +
    +        This will also perform the merging locally on each mapper before
    +        sending results to reducer, similarly to a "combiner" in MapReduce.
    +
    +        Output will be hash-partitioned with C{numPartitions} partitions, or
    +        the default parallelism level if C{numPartitions} is not specified.
    +        """
    +        return self.combineByKey(lambda x: x, func, func, numPartitions)
    +
    +    def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
    +                     numPartitions=None):
    +        """
    +        Count the number of elements for each key, and return the result to the
    +        master as a dictionary
    +        """
    +        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 partitioned using the specified partitioner.
    +        """
    +        return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc))
    +
    +    def foreach(self, func):
    +        return self.foreachRDD(lambda _, rdd: rdd.foreach(func))
    +
    +    def foreachRDD(self, func):
    +        """
    +        Apply userdefined function to all RDD in a DStream.
    +        This python implementation could be expensive because it uses callback server
    +        in order to apply function to RDD in DStream.
    +        This is an output operator, so this DStream will be registered as an output
    +        stream and there materialized.
    +        """
    +        jfunc = RDDFunction(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. This is an output
    +        operator, so this DStream will be registered as an output stream and there materialized.
    +        """
    +        def takeAndPrint(timestamp, rdd):
    +            taken = rdd.take(11)
    +            print "-------------------------------------------"
    +            print "Time: %s" % datetime.fromtimestamp(timestamp / 1000.0)
    +            print "-------------------------------------------"
    +            for record in taken[:10]:
    +                print record
    +            if len(taken) > 10:
    +                print "..."
    +            print
    +
    +        self.foreachRDD(takeAndPrint)
    +
    +    def _first(self):
    +        """
    +        Return the first RDD in the stream.
    +        """
    +        return self._take(1)[0]
    +
    +    def _take(self, n):
    +        """
    +        Return the first `n` RDDs in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        self.foreachRDD(take)
    +
    +        self._ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self._ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            result.append(r)
    +        self.foreachRDD(get_output)
    +        return result
    +
    +    def mapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a map function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        map_values_fn = lambda (k, v): (k, f(v))
    +        return self.map(map_values_fn, preservesPartitioning=True)
    +
    +    def flatMapValues(self, f):
    +        """
    +        Pass each value in the key-value pair RDD through a flatMap function
    +        without changing the keys; this also retains the original RDD's
    +        partitioning.
    +        """
    +        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. Applying glom() to an RDD coalesces all
    +        elements within each partition into an list.
    +        """
    +        def func(iterator):
    +            yield list(iterator)
    +        return self.mapPartitions(func)
    +
    +    def cache(self):
    +        """
    +        Persist 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):
    +        """
    +        Set this DStream's storage level to persist its values across operations
    +        after the first time it is computed. This can only be used to assign
    +        a new storage level if the DStream does not have a storage level set yet.
    +        """
    +        self.is_cached = True
    +        javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
    +        self._jdstream.persist(javaStorageLevel)
    +        return self
    +
    +    def checkpoint(self, interval):
    +        """
    +        Mark this DStream for checkpointing. It will be saved to a file inside the
    +        checkpoint directory set with L{SparkContext.setCheckpointDir()}
    +
    +        @param interval: time in seconds, after which 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 which contains group the values for each key in the
    +        DStream into a single sequence.
    +        Hash-partitions the resulting RDD with into numPartitions partitions in
    +        the DStream.
    +
    +        Note: If you are grouping in order to perform an aggregation (such as a
    +        sum or average) over each key, using reduceByKey will provide much
    +        better performance.
    +        """
    +        return self.transform(lambda rdd: rdd.groupByKey(numPartitions))
    +
    +    def countByValue(self):
    +        """
    +        Return new DStream which contains the count of each unique value in this
    +        DStreeam as a (value, count) pairs.
    +        """
    +        return self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None).count()
    +
    +    def saveAsTextFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a text file, using string representations of elements.
    +        """
    +
    +        def saveAsTextFile(time, rdd):
    +            """
    +            Closure to save element in RDD in DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsTextFile(path)
    +
    +        return self.foreachRDD(saveAsTextFile)
    +
    +    def saveAsPickleFiles(self, prefix, suffix=None):
    +        """
    +        Save this DStream as a SequenceFile of serialized objects. The serializer
    +        used is L{pyspark.serializers.PickleSerializer}, default batch size
    +        is 10.
    +        """
    +
    +        def saveAsPickleFile(time, rdd):
    +            """
    +            Closure to save element in RDD in the DStream as Pickled data in file.
    +            This closure is called by py4j callback server.
    +            """
    +            path = rddToFileName(prefix, suffix, time)
    +            rdd.saveAsPickleFile(path)
    +
    +        return self.foreachRDD(saveAsPickleFile)
    +
    +    def transform(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, lambda t, a: func(a), True)
    +
    +    def transformWithTime(self, func):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream.
    +        """
    +        return TransformedDStream(self, func, False)
    +
    +    def transformWith(self, func, other, keepSerializer=False):
    +        """
    +        Return a new DStream in which each RDD is generated by applying a function
    +        on each RDD of 'this' DStream and 'other' DStream.
    +        """
    +        jfunc = RDDFunction(self.ctx, lambda t, a, b: func(a, b), self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonTransformed2DStream(self._jdstream.dstream(),
    +                                                          other._jdstream.dstream(), jfunc)
    +        jrdd_serializer = self._jrdd_deserializer if keepSerializer else self.ctx.serializer
    +        return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer)
    +
    +    def repartitions(self, numPartitions):
    +        """
    +        Return a new DStream with an increased or decreased level of parallelism. Each RDD in the
    +        returned DStream has exactly numPartitions partitions.
    +        """
    +        return self.transform(lambda rdd: rdd.repartition(numPartitions))
    +
    +    @property
    +    def _slideDuration(self):
    +        """
    +        Return the slideDuration in seconds of this DStream
    +        """
    +        return self._jdstream.dstream().slideDuration().milliseconds() / 1000.0
    +
    +    def union(self, other):
    +        """
    +        Return a new DStream by unifying data of another DStream with this DStream.
    +        @param other Another DStream having the same interval (i.e., slideDuration) as this DStream.
    +        """
    +        if self._slideDuration != other._slideDuration:
    +            raise ValueError("the two DStream should have same slide duration")
    +        return self.transformWith(lambda a, b: a.union(b), other, True)
    +
    +    def cogroup(self, other, numPartitions=None):
    +        """
    +        Return a new DStream by applying 'cogroup' between RDDs of `this`
    +        DStream and `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.cogroup(b, numPartitions), other)
    +
    +    def join(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.join(b, numPartitions), other)
    +
    +    def leftOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.leftOuterJoin(b, numPartitions), other)
    +
    +    def rightOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.rightOuterJoin(b, numPartitions), other)
    +
    +    def fullOuterJoin(self, other, numPartitions=None):
    +        """
    +         Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and
    +        `other` DStream.
    +
    +        Hash partitioning is used to generate the RDDs with `numPartitions`
    +         partitions.
    +        """
    +        return self.transformWith(lambda a, b: a.fullOuterJoin(b, numPartitions), other)
    +
    +    def _jtime(self, timestamp):
    +        """ Convert datetime or unix_timestamp into Time
    +        """
    +        if isinstance(timestamp, datetime):
    +            timestamp = time.mktime(timestamp.timetuple())
    +        return self.ctx._jvm.Time(long(timestamp * 1000))
    +
    +    def slice(self, begin, end):
    +        """
    +        Return all the RDDs between 'begin' to 'end' (both included)
    +
    +        `begin`, `end` could be datetime.datetime() or unix_timestamp
    +        """
    +        jrdds = self._jdstream.slice(self._jtime(begin), self._jtime(end))
    +        return [RDD(jrdd, self.ctx, self._jrdd_deserializer) for jrdd in jrdds]
    +
    +    def _check_window(self, window, slide):
    +        duration = self._jdstream.dstream().slideDuration().milliseconds()
    +        if int(window * 1000) % duration != 0:
    +            raise ValueError("windowDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +        if slide and int(slide * 1000) % duration != 0:
    +            raise ValueError("slideDuration must be multiple of the slide duration (%d ms)"
    +                             % duration)
    +
    +    def window(self, windowDuration, slideDuration=None):
    +        """
    +        Return a new DStream in which each RDD contains all the elements in seen in a
    +        sliding window of time over this DStream.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        """
    +        self._check_window(windowDuration, slideDuration)
    +        d = self._ssc._jduration(windowDuration)
    +        if slideDuration is None:
    +            return DStream(self._jdstream.window(d), self._ssc, self._jrdd_deserializer)
    +        s = self._ssc._jduration(slideDuration)
    +        return DStream(self._jdstream.window(d, s), self._ssc, self._jrdd_deserializer)
    +
    +    def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated by reducing all
    +        elements in a sliding window over this DStream.
    +
    +        if `invReduceFunc` is not None, the reduction is done incrementally
    +        using the old window's reduced value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +         This is more efficient than `invReduceFunc` is None.
    +
    +        @param reduceFunc associative reduce function
    +        @param invReduceFunc inverse reduce function of `reduceFunc`
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        """
    +        keyed = self.map(lambda x: (1, x))
    +        reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc,
    +                                             windowDuration, slideDuration, 1)
    +        return reduced.map(lambda (k, v): v)
    +
    +    def countByWindow(self, windowDuration, slideDuration):
    +        """
    +        Return a new DStream in which each RDD has a single element generated
    +        by counting the number of elements in a window over this DStream.
    +        windowDuration and slideDuration are as defined in the window() operation.
    +
    +        This is equivalent to window(windowDuration, slideDuration).count(),
    +        but will be more efficient if window is large.
    +        """
    +        return self.map(lambda x: 1).reduceByWindow(operator.add, operator.sub,
    +                                                    windowDuration, slideDuration)
    +
    +    def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream in which each RDD contains the count of distinct elements in
    +        RDDs in a sliding window over this DStream.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  number of partitions of each RDD in the new DStream.
    +        """
    +        keyed = self.map(lambda x: (x, 1))
    +        counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub,
    +                                             windowDuration, slideDuration, numPartitions)
    +        return counted.filter(lambda (k, v): v > 0).count()
    +
    +    def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None):
    +        """
    +        Return a new DStream by applying `groupByKey` over a sliding window.
    +        Similar to `DStream.groupByKey()`, but applies it over a sliding window.
    +
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  Number of partitions of each RDD in the new DStream.
    +        """
    +        ls = self.mapValues(lambda x: [x])
    +        grouped = ls.reduceByKeyAndWindow(lambda a, b: a.extend(b) or a, lambda a, b: a[len(b):],
    +                                          windowDuration, slideDuration, numPartitions)
    +        return grouped.mapValues(ResultIterable)
    +
    +    def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None,
    +                             numPartitions=None, filterFunc=None):
    +        """
    +        Return a new DStream by applying incremental `reduceByKey` over a sliding window.
    +
    +        The reduced value of over a new window is calculated using the old window's reduce value :
    +         1. reduce the new values that entered the window (e.g., adding new counts)
    +         2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
    +
    +        `invFunc` can be None, then it will reduce all the RDDs in window, could be slower
    +        than having `invFunc`.
    +
    +        @param reduceFunc     associative reduce function
    +        @param invReduceFunc  inverse function of `reduceFunc`
    +        @param windowDuration width of the window; must be a multiple of this DStream's
    +                              batching interval
    +        @param slideDuration  sliding interval of the window (i.e., the interval after which
    +                              the new DStream will generate RDDs); must be a multiple of this
    +                              DStream's batching interval
    +        @param numPartitions  number of partitions of each RDD in the new DStream.
    +        @param filterFunc     function to filter expired key-value pairs;
    +                              only pairs that satisfy the function are retained
    +                              set this to null if you do not want to filter
    +        """
    +        self._check_window(windowDuration, slideDuration)
    +        reduced = self.reduceByKey(func)
    +
    +        def reduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            r = a.union(b).reduceByKey(func, numPartitions) if a else b
    +            if filterFunc:
    +                r = r.filter(filterFunc)
    +            return r
    +
    +        def invReduceFunc(t, a, b):
    +            b = b.reduceByKey(func, numPartitions)
    +            joined = a.leftOuterJoin(b, numPartitions)
    +            return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1)
    +
    +        jreduceFunc = RDDFunction(self.ctx, reduceFunc, reduced._jrdd_deserializer)
    +        if invReduceFunc:
    +            jinvReduceFunc = RDDFunction(self.ctx, invReduceFunc, reduced._jrdd_deserializer)
    +        else:
    +            jinvReduceFunc = None
    +        if slideDuration is None:
    +            slideDuration = self._slideDuration
    +        dstream = self.ctx._jvm.PythonReducedWindowedDStream(reduced._jdstream.dstream(),
    +                                                             jreduceFunc, jinvReduceFunc,
    +                                                             self._ssc._jduration(windowDuration),
    +                                                             self._ssc._jduration(slideDuration))
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +    def updateStateByKey(self, updateFunc, numPartitions=None):
    +        """
    +        Return a new "state" DStream where the state for each key is updated by applying
    +        the given function on the previous state of the key and the new values of the key.
    +
    +        @param updateFunc State update function ([(k, vs, s)] -> [(k, s)]).
    +                          If `s` is None, then `k` will be eliminated.
    +        """
    +        def reduceFunc(t, a, b):
    +            if a is None:
    +                g = b.groupByKey(numPartitions).map(lambda (k, vs): (k, list(vs), None))
    +            else:
    +                g = a.cogroup(b, numPartitions)
    +                g = g.map(lambda (k, (va, vb)): (k, list(vb), list(va)[0] if len(va) else None))
    +            state = g.mapPartitions(lambda x: updateFunc(x))
    +            return state.filter(lambda (k, v): v is not None)
    +
    +        jreduceFunc = RDDFunction(self.ctx, reduceFunc,
    +                                  self.ctx.serializer, self._jrdd_deserializer)
    +        dstream = self.ctx._jvm.PythonStateDStream(self._jdstream.dstream(), jreduceFunc)
    +        return DStream(dstream.asJavaDStream(), self._ssc, self.ctx.serializer)
    +
    +
    +class TransformedDStream(DStream):
    --- End diff --
    
    Please add a few lines about the design of this TransformedDStream, especially about the pipelining of transformed dstreams by squashing of transform functions.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58783286
  
    Alright, I am merging this!! This is awesome, great great work @giwa and @davies. Thank a lot to both of you, and also to @JoshRosen for ongoing help, suggestions and discussions.
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57518979
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/224/consoleFull) for   PR 2538 at commit [`bd8a4c2`](https://github.com/apache/spark/commit/bd8a4c2516147f1e99cf1f6e721346c18db23a20).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58740676
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/350/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57721916
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21226/consoleFull) for   PR 2538 at commit [`37fe06f`](https://github.com/apache/spark/commit/37fe06fb743a1934d834d603e04b678110bc0fd5).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57519132
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21141/consoleFull) for   PR 2538 at commit [`bd8a4c2`](https://github.com/apache/spark/commit/bd8a4c2516147f1e99cf1f6e721346c18db23a20).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58735252
  
    @davies unfortunately the last update to statebykey introduced some bug. Here is the error in jenkins logs.
    
    
    Traceback (most recent call last):
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/streaming/util.py", line 57, in call
        r = self.func(t, *rdds)
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/streaming/dstream.py", line 254, in saveAsTextFile
        rdd.saveAsTextFile(path)
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/rdd.py", line 1267, in saveAsTextFile
        keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path)
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/lib/py4j-0.8.2.1-src.zip/py4j/java_gateway.py", line 538, in __call__
        self.target_id, self.name)
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py", line 300, in get_return_value
        format(target_id, '.', name), value)
    Py4JJavaError: An error occurred while calling o62024.saveAsTextFile.
    : org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 296848.0 failed 1 times, most recent failure: Lost task 0.0 in stage 296848.0 (TID 780, localhost): org.apache.spark.api.python.PythonException: Traceback (most recent call last):
      File "pyspark/worker.py", line 109, in main
        process()
      File "pyspark/worker.py", line 100, in process
        serializer.dump_stream(func(split_index, iterator), outfile)
      File "pyspark/serializers.py", line 234, in dump_stream
        vs = list(itertools.islice(iterator, batch))
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/rdd.py", line 1641, in <lambda>
        map_values_fn = lambda (k, v): (k, f(v))
      File "/home/jenkins/workspace/SparkPullRequestBuilder@2/python/pyspark/streaming/dstream.py", line 579, in <lambda>
        state = g.mapValues(lambda (vs, s): updateFunc(vs, s))
    TypeError: updater() takes exactly 1 argument (2 given)
    
            org.apache.spark.api.python.PythonRDD$$anon$1.read(PythonRDD.scala:135)
            org.apache.spark.api.python.PythonRDD$$anon$1.<init>(PythonRDD.scala:166)
            org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:94)
            org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
            org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
            org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply$mcV$sp(PythonRDD.scala:235)
            org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:196)
            org.apache.spark.api.python.PythonRDD$WriterThread$$anonfun$run$1.apply(PythonRDD.scala:196)
            org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1321)
            org.apache.spark.api.python.PythonRDD$WriterThread.run(PythonRDD.scala:195)
    Driver stacktrace:
    	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1191)
    	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1180)
    	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1179)
    	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
    	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
    	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1179)
    	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:694)
    	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:694)
    	at scala.Option.foreach(Option.scala:236)
    	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:694)
    	at org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2.applyOrElse(DAGScheduler.scala:1397)
    	at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
    	at org.apache.spark.scheduler.DAGSchedulerEventProcessActor.aroundReceive(DAGScheduler.scala:1352)
    	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
    	at akka.actor.ActorCell.invoke(ActorCell.scala:487)
    	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
    	at akka.dispatch.Mailbox.run(Mailbox.scala:220)
    	at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:393)
    	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
    	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
    	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
    	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57658106
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21206/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18378352
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -52,12 +52,18 @@ private[spark] class PythonRDD(
         accumulator: Accumulator[JList[Array[Byte]]])
       extends RDD[Array[Byte]](parent) {
     
    +  // create a new PythonRDD with same Python setting but different parent.
    +  def copyTo(rdd: RDD[_]): PythonRDD = {
    --- End diff --
    
    It doesn't look like this is called from anywhere?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57709252
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/257/consoleFull) for   PR 2538 at commit [`d05871e`](https://github.com/apache/spark/commit/d05871e912ee4828a4ac68a6a0ceed0454e44722).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57762288
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21242/consoleFull) for   PR 2538 at commit [`e108ec1`](https://github.com/apache/spark/commit/e108ec114eb1a14c6e2387761da8e55bee4b3c83).
     * This patch **fails** unit tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class StreamingContext(object):`
      * `class DStream(object):`
      * `class TransformedDStream(DStream):`
      * `class TransformFunction(object):`
      * `class TransformFunctionSerializer(object):`
      * `  protected case class Keyword(str: String)`



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58739325
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/350/consoleFull) for   PR 2538 at commit [`6db00da`](https://github.com/apache/spark/commit/6db00da9595e38eccff7bfb5683b32cee3ac6263).
     * This patch merges cleanly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18079362
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,321 @@
    +#
    +# 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.
    +#
    +
    +"""
    +Unit tests for Python SparkStreaming; additional tests are implemented as doctests in
    +individual modules.
    +
    +Callback server is sometimes unstable sometimes, which cause error in test case.
    +But this is very rare case.
    +"""
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +
    +from pyspark.context import SparkContext
    +from pyspark.streaming.context import StreamingContext
    +from pyspark.streaming.duration import Seconds
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        self.sc = SparkContext(appName=class_name)
    +        self.ssc = StreamingContext(self.sc, duration=Seconds(1))
    +
    +    def tearDown(self):
    +        # Do not call pyspark.streaming.context.StreamingContext.stop directly because
    +        # we do not wait to shutdown py4j client.
    +        self.ssc.stop()
    +        self.sc.stop()
    +        time.sleep(1)
    +
    +    @classmethod
    +    def tearDownClass(cls):
    +        # Make sure tp shutdown the callback server
    +        SparkContext._gateway._shutdown_callback_server()
    +
    +
    +class TestBasicOperations(PySparkStreamingTestCase):
    +    """
    +    2 tests for each function for batach deserializer and unbatch deserilizer because
    +    the deserializer is not changed dunamically after streaming process starts.
    +    Default numInputPartitions is 2.
    +    If the number of input element is over 3, that DStream use batach deserializer.
    +    If not, that DStream use unbatch deserializer.
    +
    +    All tests input should have list of lists(3 lists are default). This list represents stream.
    +    Every batch interval, the first object of list are chosen to make DStream.
    +    e.g The first list in the list is input of the first batch.
    +    Please see the BasicTestSuits in Scala which is close to this implementation.
    +    """
    +    def setUp(self):
    +        PySparkStreamingTestCase.setUp(self)
    +        self.timeout = 10  # seconds
    +        self.numInputPartitions = 2
    +
    +    def test_map(self):
    +        """Basic operation test for DStream.map."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.map(str)
    +        expected = map(lambda x: map(str, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_flatMap(self):
    +        """Basic operation test for DStream.faltMap."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.flatMap(lambda x: (x, x * 2))
    +        expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))),
    +                       input)
    +        self._test_func(input, func, expected)
    +
    +    def test_filter(self):
    +        """Basic operation test for DStream.filter."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.filter(lambda x: x % 2 == 0)
    +        expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_count(self):
    +        """Basic operation test for DStream.count."""
    +        input = [range(1, 5), range(1, 10), range(1, 20)]
    +
    +        def func(dstream):
    +            return dstream.count()
    +        expected = map(lambda x: [len(x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduce(self):
    +        """Basic operation test for DStream.reduce."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.reduce(operator.add)
    +        expected = map(lambda x: [reduce(operator.add, x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduceByKey(self):
    +        """Basic operation test for DStream.reduceByKey."""
    +        input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)],
    +                 [("", 1), ("", 1), ("", 1), ("", 1)],
    +                 [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]]
    +
    +        def func(dstream):
    +            return dstream.reduceByKey(operator.add)
    +        expected = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_mapValues(self):
    +        """Basic operation test for DStream.mapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 2), (3, 3)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.mapValues(lambda x: x + 10)
    +        expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)],
    +                    [("", 14), (1, 11), (2, 12), (3, 13)],
    +                    [(1, 11), (2, 11), (3, 11), (4, 11)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_flatMapValues(self):
    +        """Basic operation test for DStream.flatMapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 1), (3, 1)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.flatMapValues(lambda x: (x, x + 10))
    +        expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12),
    +                     ("c", 1), ("c", 11), ("d", 1), ("d", 11)],
    +                    [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)],
    +                    [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]]
    +        self._test_func(input, func, expected)
    +
    +    def test_glom(self):
    +        """Basic operation test for DStream.glom."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        numSlices = 2
    +
    +        def func(dstream):
    +            return dstream.glom()
    +        expected = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]]
    +        self._test_func(input, func, expected, numSlices)
    +
    +    def test_mapPartitions(self):
    +        """Basic operation test for DStream.mapPartitions."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        numSlices = 2
    +
    +        def func(dstream):
    +            def f(iterator):
    +                yield sum(iterator)
    +            return dstream.mapPartitions(f)
    +        expected = [[3, 7], [11, 15], [19, 23]]
    +        self._test_func(input, func, expected, numSlices)
    +
    +    def test_countByValue(self):
    +        """Basic operation test for DStream.countByValue."""
    +        input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]]
    +
    +        def func(dstream):
    +            return dstream.countByValue()
    +        expected = [[4], [4], [3]]
    +        self._test_func(input, func, expected)
    +
    +    def test_groupByKey(self):
    +        """Basic operation test for DStream.groupByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            return dstream.groupByKey().mapValues(list)
    +
    +        expected = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])],
    +                    [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])],
    +                    [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_combineByKey(self):
    +        """Basic operation test for DStream.combineByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            def add(a, b):
    +                return a + str(b)
    +            return dstream.combineByKey(str, add, add)
    +        expected = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")],
    +                    [(1, "111"), (2, "11"), (3, "1")],
    +                    [("a", "11"), ("b", "1"), ("", "111")]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_union(self):
    +        input1 = [range(3), range(5), range(1)]
    +        input2 = [range(3, 6), range(5, 6), range(1, 6)]
    +
    +        d1 = self.ssc._makeStream(input1)
    +        d2 = self.ssc._makeStream(input2)
    +        d = d1.union(d2)
    +        result = d.collect()
    +        expected = [range(6), range(6), range(6)]
    +
    +        self.ssc.start()
    +        start_time = time.time()
    +        # Loop until get the expected the number of the result from the stream.
    +        while True:
    +            current_time = time.time()
    +            # Check time out.
    +            if (current_time - start_time) > self.timeout * 2:
    +                break
    +            # StreamingContext.awaitTermination is not used to wait because
    +            # if py4j server is called every 50 milliseconds, it gets an error.
    +            time.sleep(0.05)
    +            # Check if the output is the same length of expected output.
    +            if len(expected) == len(result):
    +                break
    +        self.assertEqual(expected, result)
    +
    +    def _sort_result_based_on_key(self, outputs):
    +        """Sort the list base onf first value."""
    --- End diff --
    
    Sorry this may be my typo.
    
    ```
    """Sort the list base onf first value."""
    ```
    based on the first value


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [WIP] [SPARK-2377] Python API for Streaming

Posted by giwa <gi...@git.apache.org>.
Github user giwa commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-56942157
  
    @davies @JoshRosen 
    Is there any idea about how to kill callback server after an error is happened?
    
    I tried signal and atexit to call gateway._shutdown_callback_server()
    This would be a critical trouble for CI. Test cases for streaming get error because callback server port is occupied by previous thread. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58251927
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21398/Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-57590494
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21181/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2538#discussion_r18318391
  
    --- Diff: python/pyspark/streaming/tests.py ---
    @@ -0,0 +1,532 @@
    +#
    +# 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.
    +#
    +
    +import os
    +from itertools import chain
    +import time
    +import operator
    +import unittest
    +import tempfile
    +
    +from pyspark.context import SparkConf, SparkContext, RDD
    +from pyspark.streaming.context import StreamingContext
    +
    +
    +class PySparkStreamingTestCase(unittest.TestCase):
    +
    +    timeout = 10  # seconds
    +    duration = 1
    +
    +    def setUp(self):
    +        class_name = self.__class__.__name__
    +        conf = SparkConf().set("spark.default.parallelism", 1)
    +        self.sc = SparkContext(appName=class_name, conf=conf)
    +        self.sc.setCheckpointDir("/tmp")
    +        # TODO: decrease duration to speed up tests
    +        self.ssc = StreamingContext(self.sc, self.duration)
    +
    +    def tearDown(self):
    +        self.ssc.stop()
    +
    +    def _take(self, dstream, n):
    +        """
    +        Return the first `n` elements in the stream (will start and stop).
    +        """
    +        results = []
    +
    +        def take(_, rdd):
    +            if rdd and len(results) < n:
    +                results.extend(rdd.take(n - len(results)))
    +
    +        dstream.foreachRDD(take)
    +
    +        self.ssc.start()
    +        while len(results) < n:
    +            time.sleep(0.01)
    +        self.ssc.stop(False, True)
    +        return results
    +
    +    def _collect(self, dstream):
    +        """
    +        Collect each RDDs into the returned list.
    +
    +        :return: list, which will have the collected items.
    +        """
    +        result = []
    +
    +        def get_output(_, rdd):
    +            r = rdd.collect()
    +            if r:
    +                result.append(r)
    +        dstream.foreachRDD(get_output)
    +        return result
    +
    +    def _test_func(self, input, func, expected, sort=False, input2=None):
    +        """
    +        @param input: dataset for the test. This should be list of lists.
    +        @param func: wrapped function. This function should return PythonDStream object.
    +        @param expected: expected output for this testcase.
    +        """
    +        if not isinstance(input[0], RDD):
    +            input = [self.sc.parallelize(d, 1) for d in input]
    +        input_stream = self.ssc.queueStream(input)
    +        if input2 and not isinstance(input2[0], RDD):
    +            input2 = [self.sc.parallelize(d, 1) for d in input2]
    +        input_stream2 = self.ssc.queueStream(input2) if input2 is not None else None
    +
    +        # Apply test function to stream.
    +        if input2:
    +            stream = func(input_stream, input_stream2)
    +        else:
    +            stream = func(input_stream)
    +
    +        result = self._collect(stream)
    +        self.ssc.start()
    +
    +        start_time = time.time()
    +        # Loop until get the expected the number of the result from the stream.
    +        while True:
    +            current_time = time.time()
    +            # Check time out.
    +            if (current_time - start_time) > self.timeout:
    +                print "timeout after", self.timeout
    +                break
    +            # StreamingContext.awaitTermination is not used to wait because
    +            # if py4j server is called every 50 milliseconds, it gets an error.
    +            time.sleep(0.05)
    +            # Check if the output is the same length of expected output.
    +            if len(expected) == len(result):
    +                break
    +        if sort:
    +            self._sort_result_based_on_key(result)
    +            self._sort_result_based_on_key(expected)
    +        self.assertEqual(expected, result)
    +
    +    def _sort_result_based_on_key(self, outputs):
    +        """Sort the list based on first value."""
    +        for output in outputs:
    +            output.sort(key=lambda x: x[0])
    +
    +
    +class TestBasicOperations(PySparkStreamingTestCase):
    +
    +    def test_map(self):
    +        """Basic operation test for DStream.map."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.map(str)
    +        expected = map(lambda x: map(str, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_flatMap(self):
    +        """Basic operation test for DStream.faltMap."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.flatMap(lambda x: (x, x * 2))
    +        expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))),
    +                       input)
    +        self._test_func(input, func, expected)
    +
    +    def test_filter(self):
    +        """Basic operation test for DStream.filter."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.filter(lambda x: x % 2 == 0)
    +        expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input)
    +        self._test_func(input, func, expected)
    +
    +    def test_count(self):
    +        """Basic operation test for DStream.count."""
    +        input = [range(5), range(10), range(20)]
    +
    +        def func(dstream):
    +            return dstream.count()
    +        expected = map(lambda x: [len(x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduce(self):
    +        """Basic operation test for DStream.reduce."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +
    +        def func(dstream):
    +            return dstream.reduce(operator.add)
    +        expected = map(lambda x: [reduce(operator.add, x)], input)
    +        self._test_func(input, func, expected)
    +
    +    def test_reduceByKey(self):
    +        """Basic operation test for DStream.reduceByKey."""
    +        input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)],
    +                 [("", 1), ("", 1), ("", 1), ("", 1)],
    +                 [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]]
    +
    +        def func(dstream):
    +            return dstream.reduceByKey(operator.add)
    +        expected = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_mapValues(self):
    +        """Basic operation test for DStream.mapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 2), (3, 3)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.mapValues(lambda x: x + 10)
    +        expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)],
    +                    [("", 14), (1, 11), (2, 12), (3, 13)],
    +                    [(1, 11), (2, 11), (3, 11), (4, 11)]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_flatMapValues(self):
    +        """Basic operation test for DStream.flatMapValues."""
    +        input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)],
    +                 [("", 4), (1, 1), (2, 1), (3, 1)],
    +                 [(1, 1), (2, 1), (3, 1), (4, 1)]]
    +
    +        def func(dstream):
    +            return dstream.flatMapValues(lambda x: (x, x + 10))
    +        expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12),
    +                     ("c", 1), ("c", 11), ("d", 1), ("d", 11)],
    +                    [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)],
    +                    [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]]
    +        self._test_func(input, func, expected)
    +
    +    def test_glom(self):
    +        """Basic operation test for DStream.glom."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            return dstream.glom()
    +        expected = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_mapPartitions(self):
    +        """Basic operation test for DStream.mapPartitions."""
    +        input = [range(1, 5), range(5, 9), range(9, 13)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            def f(iterator):
    +                yield sum(iterator)
    +            return dstream.mapPartitions(f)
    +        expected = [[3, 7], [11, 15], [19, 23]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_countByValue(self):
    +        """Basic operation test for DStream.countByValue."""
    +        input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]]
    +
    +        def func(dstream):
    +            return dstream.countByValue()
    +        expected = [[4], [4], [3]]
    +        self._test_func(input, func, expected)
    +
    +    def test_groupByKey(self):
    +        """Basic operation test for DStream.groupByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            return dstream.groupByKey().mapValues(list)
    +
    +        expected = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])],
    +                    [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])],
    +                    [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_combineByKey(self):
    +        """Basic operation test for DStream.combineByKey."""
    +        input = [[(1, 1), (2, 1), (3, 1), (4, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]]
    +
    +        def func(dstream):
    +            def add(a, b):
    +                return a + str(b)
    +            return dstream.combineByKey(str, add, add)
    +        expected = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")],
    +                    [(1, "111"), (2, "11"), (3, "1")],
    +                    [("a", "11"), ("b", "1"), ("", "111")]]
    +        self._test_func(input, func, expected, sort=True)
    +
    +    def test_repartition(self):
    +        input = [range(1, 5), range(5, 9)]
    +        rdds = [self.sc.parallelize(r, 2) for r in input]
    +
    +        def func(dstream):
    +            return dstream.repartition(1).glom()
    +        expected = [[[1, 2, 3, 4]], [[5, 6, 7, 8]]]
    +        self._test_func(rdds, func, expected)
    +
    +    def test_union(self):
    +        input1 = [range(3), range(5), range(6)]
    +        input2 = [range(3, 6), range(5, 6)]
    +
    +        def func(d1, d2):
    +            return d1.union(d2)
    +
    +        expected = [range(6), range(6), range(6)]
    +        self._test_func(input1, func, expected, input2=input2)
    +
    +    def test_cogroup(self):
    +        input = [[(1, 1), (2, 1), (3, 1)],
    +                 [(1, 1), (1, 1), (1, 1), (2, 1)],
    +                 [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1)]]
    +        input2 = [[(1, 2)],
    +                  [(4, 1)],
    +                  [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 2)]]
    +
    +        def func(d1, d2):
    +            return d1.cogroup(d2).mapValues(lambda vs: tuple(map(list, vs)))
    +
    +        expected = [[(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))],
    +                    [(1, ([1, 1, 1], [])), (2, ([1], [])), (4, ([], [1]))],
    +                    [("a", ([1, 1], [1, 1])), ("b", ([1], [1])), ("", ([1, 1], [1, 2]))]]
    +        self._test_func(input, func, expected, sort=True, input2=input2)
    +
    +    def test_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.join(b)
    +
    +        expected = [[('b', (2, 3))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_left_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.leftOuterJoin(b)
    +
    +        expected = [[('a', (1, None)), ('b', (2, 3))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_right_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.rightOuterJoin(b)
    +
    +        expected = [[('b', (2, 3)), ('c', (None, 4))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def test_full_outer_join(self):
    +        input = [[('a', 1), ('b', 2)]]
    +        input2 = [[('b', 3), ('c', 4)]]
    +
    +        def func(a, b):
    +            return a.fullOuterJoin(b)
    +
    +        expected = [[('a', (1, None)), ('b', (2, 3)), ('c', (None, 4))]]
    +        self._test_func(input, func, expected, True, input2)
    +
    +    def update_state_by_key(self):
    +
    +        def updater(it):
    +            for k, vs, s in it:
    +                if not s:
    +                    s = vs
    +                else:
    +                    s.extend(vs)
    +                yield (k, s)
    +
    +        input = [[('k', i)] for i in range(5)]
    +
    +        def func(dstream):
    +            return dstream.updateStateByKey(updater)
    +
    +        expected = [[0], [0, 1], [0, 1, 2], [0, 1, 2, 3], [0, 1, 2, 3, 4]]
    +        expected = [[('k', v)] for v in expected]
    +        self._test_func(input, func, expected)
    +
    +
    +class TestWindowFunctions(PySparkStreamingTestCase):
    +
    +    timeout = 20
    +
    +    def test_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5)]
    +
    +        def func(dstream):
    +            return dstream.window(3, 1).count()
    +
    +        expected = [[1], [3], [6], [9], [12], [9], [5]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5)]
    +
    +        def func(dstream):
    +            return dstream.countByWindow(3, 1)
    +
    +        expected = [[1], [3], [6], [9], [12], [9], [5]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_window_large(self):
    +        input = [range(1), range(2), range(3), range(4), range(5), range(6)]
    +
    +        def func(dstream):
    +            return dstream.countByWindow(5, 1)
    +
    +        expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]]
    +        self._test_func(input, func, expected)
    +
    +    def test_count_by_value_and_window(self):
    +        input = [range(1), range(2), range(3), range(4), range(5), range(6)]
    +
    +        def func(dstream):
    +            return dstream.countByValueAndWindow(5, 1)
    +
    +        expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]]
    +        self._test_func(input, func, expected)
    +
    +    def test_group_by_key_and_window(self):
    +        input = [[('a', i)] for i in range(5)]
    +
    +        def func(dstream):
    +            return dstream.groupByKeyAndWindow(3, 1).mapValues(list)
    +
    +        expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])],
    +                    [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]]
    +        self._test_func(input, func, expected)
    +
    +    def test_reduce_by_invalid_window(self):
    +        input1 = [range(3), range(5), range(1), range(6)]
    +        d1 = self.ssc.queueStream(input1)
    +        self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 0.1, 0.1))
    +        self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 1, 0.1))
    +
    +
    +class TestStreamingContext(PySparkStreamingTestCase):
    +
    +    duration = 0.1
    +
    +    def test_stop_only_streaming_context(self):
    +        self._addInputStream()
    +        self.ssc.start()
    +        self.ssc.stop(False)
    +        self.assertEqual(len(self.sc.parallelize(range(5), 5).glom().collect()), 5)
    +
    +    def test_stop_multiple_times(self):
    +        self._addInputStream()
    +        self.ssc.start()
    +        self.ssc.stop()
    +        self.ssc.stop()
    +
    +    def _addInputStream(self):
    --- End diff --
    
    nit: move this function either before or above all the tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-2377] Python API for Streaming

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2538#issuecomment-58741206
  
    **[Tests timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21627/consoleFull)**     for PR 2538 at commit [`331ecce`](https://github.com/apache/spark/commit/331ecced6f61ad5183da5830f94f584bcc74e479)     after a configured wait of `120m`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org