aboutsummaryrefslogtreecommitdiff
path: root/core/src
diff options
context:
space:
mode:
authorDaniel Darabos <darabos.daniel@gmail.com>2015-07-16 08:16:54 +0100
committerSean Owen <sowen@cloudera.com>2015-07-16 08:16:54 +0100
commit011551620faa87107a787530f074af3d9be7e695 (patch)
tree23d59a693288f5897e85b2834375a6f8cd474737 /core/src
parent0a795336df20c7ec969366e613286f0c060a4eeb (diff)
downloadspark-011551620faa87107a787530f074af3d9be7e695.tar.gz
spark-011551620faa87107a787530f074af3d9be7e695.tar.bz2
spark-011551620faa87107a787530f074af3d9be7e695.zip
[SPARK-8893] Add runtime checks against non-positive number of partitions
https://issues.apache.org/jira/browse/SPARK-8893 > What does `sc.parallelize(1 to 3).repartition(p).collect` return? I would expect `Array(1, 2, 3)` regardless of `p`. But if `p` < 1, it returns `Array()`. I think instead it should throw an `IllegalArgumentException`. > I think the case is pretty clear for `p` < 0. But the behavior for `p` = 0 is also error prone. In fact that's how I found this strange behavior. I used `rdd.repartition(a/b)` with positive `a` and `b`, but `a/b` was rounded down to zero and the results surprised me. I'd prefer an exception instead of unexpected (corrupt) results. Author: Daniel Darabos <darabos.daniel@gmail.com> Closes #7285 from darabos/patch-1 and squashes the following commits: decba82 [Daniel Darabos] Allow repartitioning empty RDDs to zero partitions. 97de852 [Daniel Darabos] Allow zero partition count in HashPartitioner f6ba5fb [Daniel Darabos] Use require() for simpler syntax. d5e3df8 [Daniel Darabos] Require positive number of partitions in HashPartitioner 897c628 [Daniel Darabos] Require positive maxPartitions in CoalescedRDD
Diffstat (limited to 'core/src')
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala5
2 files changed, 6 insertions, 1 deletions
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 82889bcd30..ad68512dcc 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -76,6 +76,8 @@ object Partitioner {
* produce an unexpected or incorrect result.
*/
class HashPartitioner(partitions: Int) extends Partitioner {
+ require(partitions >= 0, s"Number of partitions ($partitions) cannot be negative.")
+
def numPartitions: Int = partitions
def getPartition(key: Any): Int = key match {
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index 663eebb8e4..90d9735cb3 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -69,7 +69,7 @@ private[spark] case class CoalescedRDDPartition(
* the preferred location of each new partition overlaps with as many preferred locations of its
* parent partitions
* @param prev RDD to be coalesced
- * @param maxPartitions number of desired partitions in the coalesced RDD
+ * @param maxPartitions number of desired partitions in the coalesced RDD (must be positive)
* @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance
*/
private[spark] class CoalescedRDD[T: ClassTag](
@@ -78,6 +78,9 @@ private[spark] class CoalescedRDD[T: ClassTag](
balanceSlack: Double = 0.10)
extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies
+ require(maxPartitions > 0 || maxPartitions == prev.partitions.length,
+ s"Number of partitions ($maxPartitions) must be positive.")
+
override def getPartitions: Array[Partition] = {
val pc = new PartitionCoalescer(maxPartitions, prev, balanceSlack)