You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2014/04/25 08:08:01 UTC

git commit: SPARK-1242 Add aggregate to python rdd

Repository: spark
Updated Branches:
  refs/heads/master 095b51825 -> e03bc379e


SPARK-1242 Add aggregate to python rdd

Author: Holden Karau <ho...@pigscanfly.ca>

Closes #139 from holdenk/add_aggregate_to_python_api and squashes the following commits:

0f39ae3 [Holden Karau] Merge in master
4879c75 [Holden Karau] CR feedback, fix issue with empty RDDs in aggregate
70b4724 [Holden Karau] Style fixes from code review
96b047b [Holden Karau] Add aggregate to python rdd


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e03bc379
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e03bc379
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e03bc379

Branch: refs/heads/master
Commit: e03bc379ee03fde0ee4fa578d3c39aae35c63f01
Parents: 095b518
Author: Holden Karau <ho...@pigscanfly.ca>
Authored: Thu Apr 24 23:07:54 2014 -0700
Committer: Matei Zaharia <ma...@databricks.com>
Committed: Thu Apr 24 23:07:54 2014 -0700

----------------------------------------------------------------------
 python/pyspark/rdd.py | 31 +++++++++++++++++++++++++++++--
 1 file changed, 29 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/e03bc379/python/pyspark/rdd.py
----------------------------------------------------------------------
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index d73ab70..a59778c 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -599,7 +599,7 @@ class RDD(object):
     def reduce(self, f):
         """
         Reduces the elements of this RDD using the specified commutative and
-        associative binary operator.
+        associative binary operator. Currently reduces partitions locally.
 
         >>> from operator import add
         >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add)
@@ -641,7 +641,34 @@ class RDD(object):
         vals = self.mapPartitions(func).collect()
         return reduce(op, vals, zeroValue)
 
-    # TODO: aggregate
+    def aggregate(self, zeroValue, seqOp, combOp):
+        """
+        Aggregate the elements of each partition, and then the results for all
+        the partitions, using a given combine functions and a neutral "zero
+        value."
+
+        The functions C{op(t1, t2)} is allowed to modify C{t1} and return it
+        as its result value to avoid object allocation; however, it should not
+        modify C{t2}.
+
+        The first function (seqOp) can return a different result type, U, than
+        the type of this RDD. Thus, we need one operation for merging a T into an U
+        and one operation for merging two U
+
+        >>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1))
+        >>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1]))
+        >>> sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp)
+        (10, 4)
+        >>> sc.parallelize([]).aggregate((0, 0), seqOp, combOp)
+        (0, 0)
+        """
+        def func(iterator):
+            acc = zeroValue
+            for obj in iterator:
+                acc = seqOp(acc, obj)
+            yield acc
+
+        return self.mapPartitions(func).fold(zeroValue, combOp)
         
 
     def max(self):