aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark
diff options
context:
space:
mode:
authorSean Owen <sowen@cloudera.com>2016-02-19 10:26:38 +0000
committerSean Owen <sowen@cloudera.com>2016-02-19 10:26:38 +0000
commitfb7e21797ed618d9754545a44f8f95f75b66757a (patch)
tree42b592cf1f25aeaf067c35afd75f9a3403182b99 /python/pyspark
parentc776fce99b496a789ffcf2cfab78cf51eeea032b (diff)
downloadspark-fb7e21797ed618d9754545a44f8f95f75b66757a.tar.gz
spark-fb7e21797ed618d9754545a44f8f95f75b66757a.tar.bz2
spark-fb7e21797ed618d9754545a44f8f95f75b66757a.zip
[SPARK-13339][DOCS] Clarify commutative / associative operator requirements for reduce, fold
Clarify that reduce functions need to be commutative, and fold functions do not See https://github.com/apache/spark/pull/11091 Author: Sean Owen <sowen@cloudera.com> Closes #11217 from srowen/SPARK-13339.
Diffstat (limited to 'python/pyspark')
-rw-r--r--python/pyspark/rdd.py7
-rw-r--r--python/pyspark/streaming/dstream.py4
2 files changed, 5 insertions, 6 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index fe2264a63c..4eaf589ad5 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -844,8 +844,7 @@ class RDD(object):
def fold(self, zeroValue, op):
"""
Aggregate the elements of each partition, and then the results for all
- the partitions, using a given associative and commutative function and
- a neutral "zero value."
+ the partitions, using a given associative function and a neutral "zero value."
The function 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
@@ -1558,7 +1557,7 @@ class RDD(object):
def reduceByKey(self, func, numPartitions=None, partitionFunc=portable_hash):
"""
- Merge the values for each key using an associative reduce function.
+ Merge the values for each key using an associative and commutative reduce function.
This will also perform the merging locally on each mapper before
sending results to a reducer, similarly to a "combiner" in MapReduce.
@@ -1576,7 +1575,7 @@ class RDD(object):
def reduceByKeyLocally(self, func):
"""
- Merge the values for each key using an associative reduce function, but
+ Merge the values for each key using an associative and commutative reduce function, but
return the results immediately to the master as a dictionary.
This will also perform the merging locally on each mapper before
diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py
index 86447f5e58..2056663872 100644
--- a/python/pyspark/streaming/dstream.py
+++ b/python/pyspark/streaming/dstream.py
@@ -453,7 +453,7 @@ class DStream(object):
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 reduceFunc: associative and commutative 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
@@ -524,7 +524,7 @@ class DStream(object):
`invFunc` can be None, then it will reduce all the RDDs in window, could be slower
than having `invFunc`.
- @param func: associative reduce function
+ @param func: associative and commutative reduce function
@param invFunc: inverse function of `reduceFunc`
@param windowDuration: width of the window; must be a multiple of this DStream's
batching interval