You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "APeng Zhang (JIRA)" <ji...@apache.org> on 2017/07/14 02:04:00 UTC

[jira] [Created] (SPARK-21410) In RangePartitioner(partitions: Int, rdd: RDD[]), RangePartitioner.numPartitions is wrong if the number of elements in RDD (rdd.count()) is less than number of partitions (partitions in constructor).

APeng Zhang created SPARK-21410:
-----------------------------------

             Summary: In RangePartitioner(partitions: Int, rdd: RDD[]), RangePartitioner.numPartitions is wrong if the number of elements in RDD (rdd.count()) is less than number of partitions (partitions in constructor).
                 Key: SPARK-21410
                 URL: https://issues.apache.org/jira/browse/SPARK-21410
             Project: Spark
          Issue Type: Bug
          Components: Spark Core
    Affects Versions: 2.1.0, 2.0.0, 2.2.0
            Reporter: APeng Zhang
            Priority: Minor


In RangePartitioner(partitions: Int, rdd: RDD[]), RangePartitioner.numPartitions is wrong if the number of elements in RDD (rdd.count()) is less than number of partitions (partitions in constructor).
Code1 to reproduce:

{code:java}
    import spark.implicits._
    val ds = spark.createDataset(Seq((1, 1)))
    println(ds.sort("_1").rdd.getNumPartitions)
    // The output of println is 2
{code}


Code2 to reproduce:

{code:java}
  test("Number of elements in RDD is less than number of partitions") {
    val rdd = sc.parallelize(1 to 3).map(x => (x, x))
    val partitioner = new RangePartitioner(22, rdd)
    assert(partitioner.numPartitions === 3)
  }
{code}

  This test will be failed because partitioner.numPartitions is 4.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org