Skip to content

Commit 97de852

Browse files
committed
Allow zero partition count in HashPartitioner
There are valid cases where 0-size HashPartitioners are created (such as running groupByKey on an empty RDD). As long as they don't call getPartition there is nothing wrong with this. getPartition will try to divide by zero when it is called in this case, so there is no risk of silent mistakes. For negative partition counts getPartition would return bogus results, so the assertion against that remains.
1 parent f6ba5fb commit 97de852

File tree

1 file changed

+1
-1
lines changed

1 file changed

+1
-1
lines changed

core/src/main/scala/org/apache/spark/Partitioner.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ object Partitioner {
7676
* produce an unexpected or incorrect result.
7777
*/
7878
class HashPartitioner(partitions: Int) extends Partitioner {
79-
require(partitions > 0, s"Number of partitions ($partitions) must be positive.")
79+
require(partitions >= 0, s"Number of partitions ($partitions) cannot be negative.")
8080

8181
def numPartitions: Int = partitions
8282

0 commit comments

Comments
 (0)