diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6ad5ab2a2d1a..14af4670ed86 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -353,7 +353,7 @@ def func(iterator): return ifilter(f, iterator) return self.mapPartitions(func, True) - def distinct(self): + def distinct(self, numPartitions=None): """ Return a new RDD containing the distinct elements in this RDD. @@ -361,7 +361,7 @@ def distinct(self): [1, 2, 3] """ return self.map(lambda x: (x, None)) \ - .reduceByKey(lambda x, _: x) \ + .reduceByKey(lambda x, _: x, numPartitions) \ .map(lambda (x, _): x) def sample(self, withReplacement, fraction, seed=None): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index fc9310fef318..eac55cbe1519 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1694,8 +1694,11 @@ def coalesce(self, numPartitions, shuffle=False): rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) return SchemaRDD(rdd, self.sql_ctx) - def distinct(self): - rdd = self._jschema_rdd.distinct() + def distinct(self, numPartitions=None): + if numPartitions is None: + rdd = self._jschema_rdd.distinct() + else: + rdd = self._jschema_rdd.distinct(numPartitions) return SchemaRDD(rdd, self.sql_ctx) def intersection(self, other): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b687d695b01c..4ed23e083bd0 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -586,6 +586,14 @@ def test_repartitionAndSortWithinPartitions(self): self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + def test_distinct(self): + rdd = self.sc.parallelize((1, 2, 3)*10, 10) + self.assertEquals(rdd.getNumPartitions(), 10) + self.assertEquals(rdd.distinct().count(), 3) + result = rdd.distinct(5) + self.assertEquals(result.getNumPartitions(), 5) + self.assertEquals(result.count(), 3) + class TestSQL(PySparkTestCase): @@ -635,6 +643,15 @@ def test_basic_functions(self): srdd.count() srdd.collect() + def test_distinct(self): + rdd = self.sc.parallelize(['{"a": 1}', '{"b": 2}', '{"c": 3}']*10, 10) + srdd = self.sqlCtx.jsonRDD(rdd) + self.assertEquals(srdd.getNumPartitions(), 10) + self.assertEquals(srdd.distinct().count(), 3) + result = srdd.distinct(5) + self.assertEquals(result.getNumPartitions(), 5) + self.assertEquals(result.count(), 3) + class TestIO(PySparkTestCase):