aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/rdd.py
diff options
context:
space:
mode:
Diffstat (limited to 'python/pyspark/rdd.py')
-rw-r--r--python/pyspark/rdd.py12
1 files changed, 12 insertions, 0 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index bb4d035edc..65f63153cd 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -549,6 +549,18 @@ class RDD(object):
.mapPartitions(mapFunc,preservesPartitioning=True)
.flatMap(lambda x: x, preservesPartitioning=True))
+ def sortBy(self, keyfunc, ascending=True, numPartitions=None):
+ """
+ Sorts this RDD by the given keyfunc
+
+ >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
+ >>> sc.parallelize(tmp).sortBy(lambda x: x[0]).collect()
+ [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)]
+ >>> sc.parallelize(tmp).sortBy(lambda x: x[1]).collect()
+ [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
+ """
+ return self.keyBy(keyfunc).sortByKey(ascending, numPartitions).values()
+
def glom(self):
"""
Return an RDD created by coalescing all elements within each partition