Skip to content

Commit

Permalink
Merge commit 'refs/pull/6081/head' of github.com:apache/spark into range
Browse files Browse the repository at this point in the history
  • Loading branch information
Davies Liu committed May 18, 2015
2 parents 3399055 + cbf5200 commit 4590208
Show file tree
Hide file tree
Showing 3 changed files with 143 additions and 0 deletions.
72 changes: 72 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -697,6 +697,78 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
}

/**
* Creates a new RDD[Long] containing elements from `start` to `end`(exclusive), increased by
* `step` every element.
*
* @note if we need to cache this RDD, we should make sure each partition does not exceed limit.
*
* @param start the start value.
* @param end the end value.
* @param step the incremental step
* @param numSlices the partition number of the new RDD.
* @return
*/
def range(
start: Long,
end: Long,
step: Long = 1,
numSlices: Int = defaultParallelism): RDD[Long] = withScope {
assertNotStopped()
// when step is 0, range will run infinitely
require(step != 0, "step cannot be 0")
val numElements: BigInt = {
val safeStart = BigInt(start)
val safeEnd = BigInt(end)
if ((safeEnd - safeStart) % step == 0 || safeEnd > safeStart ^ step > 0) {
(safeEnd - safeStart) / step
} else {
// the remainder has the same sign with range, could add 1 more
(safeEnd - safeStart) / step + 1
}
}
parallelize(0 until numSlices, numSlices).mapPartitionsWithIndex((i, _) => {
val partitionStart = (i * numElements) / numSlices * step + start
val partitionEnd = (((i + 1) * numElements) / numSlices) * step + start
def getSafeMargin(bi: BigInt): Long =
if (bi.isValidLong) {
bi.toLong
} else if (bi > 0) {
Long.MaxValue
} else {
Long.MinValue
}
val safePartitionStart = getSafeMargin(partitionStart)
val safePartitionEnd = getSafeMargin(partitionEnd)

new Iterator[Long] {
private[this] var number: Long = safePartitionStart
private[this] var overflow: Boolean = false

override def hasNext =
if (!overflow) {
if (step > 0) {
number < safePartitionEnd
} else {
number > safePartitionEnd
}
} else false

override def next() = {
val ret = number
number += step
if (number < ret ^ step < 0) {
// we have Long.MaxValue + Long.MaxValue < Long.MaxValue
// and Long.MinValue + Long.MinValue > Long.MinValue, so iff the step causes a step
// back, we are pretty sure that we have an overflow.
overflow = true
}
ret
}
}
})
}

/** Distribute a local Scala collection to form an RDD.
*
* This method is identical to `parallelize`.
Expand Down
31 changes: 31 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -684,6 +684,37 @@ class SQLContext(@transient val sparkContext: SparkContext)
catalog.unregisterTable(Seq(tableName))
}

/**
* :: Experimental ::
* Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements
* in an range from `start` to `end`(exclusive) with step value 1.
*
* @since 1.4.0
* @group dataframe
*/
@Experimental
def range(start: Long, end: Long): DataFrame = {
createDataFrame(
sparkContext.range(start, end).map(Row(_)),
StructType(StructField("id", LongType, nullable = false) :: Nil))
}

/**
* :: Experimental ::
* Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements
* in an range from `start` to `end`(exclusive) with an step value, with partition number
* specified.
*
* @since 1.4.0
* @group dataframe
*/
@Experimental
def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = {
createDataFrame(
sparkContext.range(start, end, step, numPartitions).map(Row(_)),
StructType(StructField("id", LongType, nullable = false) :: Nil))
}

/**
* Executes a SQL query using Spark, returning the result as a [[DataFrame]]. The dialect that is
* used for SQL parsing can be configured with 'spark.sql.dialect'.
Expand Down
40 changes: 40 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -532,4 +532,44 @@ class DataFrameSuite extends QueryTest {
val p = df.logicalPlan.asInstanceOf[Project].child.asInstanceOf[Project]
assert(!p.child.isInstanceOf[Project])
}

test("SPARK-7150 range api") {
// numSlice is greater than length
val res1 = TestSQLContext.range(0, 10, 1, 15).select("id")
assert(res1.count == 10)
assert(res1.agg(sum("id")).as("sumid").collect() === Seq(Row(45)))

val res2 = TestSQLContext.range(3, 15, 3, 2).select("id")
assert(res2.count == 4)
assert(res2.agg(sum("id")).as("sumid").collect() === Seq(Row(30)))

val res3 = TestSQLContext.range(1, -2).select("id")
assert(res3.count == 0)

// start is positive, end is negative, step is negative
val res4 = TestSQLContext.range(1, -2, -2, 6).select("id")
assert(res4.count == 2)
assert(res4.agg(sum("id")).as("sumid").collect() === Seq(Row(0)))

// start, end, step are negative
val res5 = TestSQLContext.range(-3, -8, -2, 1).select("id")
assert(res5.count == 3)
assert(res5.agg(sum("id")).as("sumid").collect() === Seq(Row(-15)))

// start, end are negative, step is positive
val res6 = TestSQLContext.range(-8, -4, 2, 1).select("id")
assert(res6.count == 2)
assert(res6.agg(sum("id")).as("sumid").collect() === Seq(Row(-14)))

val res7 = TestSQLContext.range(-10, -9, -20, 1).select("id")
assert(res7.count == 0)

val res8 = TestSQLContext.range(Long.MinValue, Long.MaxValue, Long.MaxValue, 100).select("id")
assert(res8.count == 3)
assert(res8.agg(sum("id")).as("sumid").collect() === Seq(Row(-3)))

val res9 = TestSQLContext.range(Long.MaxValue, Long.MinValue, Long.MinValue, 100).select("id")
assert(res9.count == 2)
assert(res9.agg(sum("id")).as("sumid").collect() === Seq(Row(Long.MaxValue - 1)))
}
}

0 comments on commit 4590208

Please sign in to comment.