Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-7980] [SQL] Support SQLContext.range(end) #6609

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 10 additions & 2 deletions python/pyspark/sql/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ def udf(self):
return UDFRegistration(self)

@since(1.4)
def range(self, start, end, step=1, numPartitions=None):
def range(self, start, end=None, step=1, numPartitions=None):
"""
Create a :class:`DataFrame` with single LongType column named `id`,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should document the behavior when only start is declared.

containing elements in a range from `start` to `end` (exclusive) with
Expand All @@ -142,10 +142,18 @@ def range(self, start, end, step=1, numPartitions=None):

>>> sqlContext.range(1, 7, 2).collect()
[Row(id=1), Row(id=3), Row(id=5)]

>>> sqlContext.range(3).collect()
[Row(id=0), Row(id=1), Row(id=2)]
"""
if numPartitions is None:
numPartitions = self._sc.defaultParallelism
jdf = self._ssql_ctx.range(int(start), int(end), int(step), int(numPartitions))

if end is None:
jdf = self._ssql_ctx.range(0, int(start), int(step), int(numPartitions))
else:
jdf = self._ssql_ctx.range(int(start), int(end), int(step), int(numPartitions))

return DataFrame(jdf, self)

@ignore_unicode_prefix
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/sql/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,8 @@ def test_range(self):
self.assertEqual(self.sqlCtx.range(1, 1).count(), 0)
self.assertEqual(self.sqlCtx.range(1, 0, -1).count(), 1)
self.assertEqual(self.sqlCtx.range(0, 1 << 40, 1 << 39).count(), 2)
self.assertEqual(self.sqlCtx.range(-2).count(), 0)
self.assertEqual(self.sqlCtx.range(3).count(), 3)

def test_explode(self):
from pyspark.sql.functions import explode
Expand Down
11 changes: 11 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 @@ -717,6 +717,17 @@ class SQLContext(@transient val sparkContext: SparkContext)
StructType(StructField("id", LongType, nullable = false) :: Nil))
}

/**
* :: Experimental ::
* Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements
* in an range from 0 to `end`(exclusive) with step value 1.
*
* @since 1.4.0
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this is 1.4.1 now

* @group dataframe
*/
@Experimental
def range(end: Long): DataFrame = range(0, end)

/**
* :: Experimental ::
* Creates a [[DataFrame]] with a single [[LongType]] column named `id`, containing elements
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -576,5 +576,13 @@ class DataFrameSuite extends QueryTest {
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)))

// only end provided as argument
val res10 = TestSQLContext.range(10).select("id")
assert(res10.count == 10)
assert(res10.agg(sum("id")).as("sumid").collect() === Seq(Row(45)))

val res11 = TestSQLContext.range(-1).select("id")
assert(res11.count == 0)
}
}