-
Notifications
You must be signed in to change notification settings - Fork 28.2k
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-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions #6981
Changes from 8 commits
d0e2f99
5ebb235
4d8049b
638596f
849fb41
c739788
b680db6
a5ea120
02efc5d
356df78
3bfac90
5fe74e1
a8edebd
f120415
eb6760d
5a105d9
7bc9d93
d9f8ac3
6f5d95c
f3e7a9f
7d9f0eb
10e4ad1
ccb723c
c42b444
ad17e96
f775f39
1a436c9
4fb66da
740af0e
1358cdc
ec87c69
0852655
1b2e540
b382267
d6aa14e
e223bc0
56c4a92
d01b977
2259299
523542d
0ad6db8
746b80a
cdfae27
fb98ba0
3c6ae2e
70238e0
ea6c110
4afc09c
6e0c78f
5983dcc
256c357
3e095ba
bb567b6
f7b4c8c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -486,6 +486,96 @@ def ntile(n): | |
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.ntile(int(n))) | ||
|
||
@since(1.5) | ||
def dateFormat(dateCol, formatCol): | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @rxin camel case or underscore? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. underscore |
||
""" | ||
Convert the given date into the format specified by the second argument. Return type is always string. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(dateFormat('a', 'MM/dd/yyy').alias('date')).collect() | ||
[Row(date=u'04/08/2015')] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.dateFormat(dateCol, formatCol)) | ||
|
||
@since(1.5) | ||
def year(col): | ||
""" | ||
Extract the year of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(year('a').alias('year')).collect() | ||
[Row(year=2015)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.year(col)) | ||
|
||
@since(1.5) | ||
def quarter(col): | ||
""" | ||
Extract the quarter of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(quarter('a').alias('quarter')).collect() | ||
[Row(quarter=2)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.quarter(col)) | ||
|
||
@since(1.5) | ||
def month(col): | ||
""" | ||
Extract the month of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(month('a').alias('month')).collect() | ||
[Row(month=4)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.month(col)) | ||
|
||
@since(1.5) | ||
def day(col): | ||
""" | ||
Extract the day of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(day('a').alias('day')).collect() | ||
[Row(day=8)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.day(col)) | ||
|
||
@since(1.5) | ||
def hour(col): | ||
""" | ||
Extract the hours of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']).select(hour('a').alias('hour')).collect() | ||
[Row(hour=13)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.hour(col)) | ||
|
||
@since(1.5) | ||
def minute(col): | ||
""" | ||
Extract the minutes of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']).select(minute('a').alias('minute')).collect() | ||
[Row(minute=8)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.minute(col)) | ||
|
||
@since(1.5) | ||
def second(col): | ||
""" | ||
Extract the seconds of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']).select(second('a').alias('second')).collect() | ||
[Row(second=15)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.second(col)) | ||
|
||
@since(1.5) | ||
def weekOfYear(col): | ||
""" | ||
Extract the week number of a given date as integer. | ||
>>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(weekOfYear('a').alias('week')).collect() | ||
[Row(week=15)] | ||
""" | ||
sc = SparkContext._active_spark_context | ||
return Column(sc._jvm.functions.weekOfYear(col)) | ||
|
||
|
||
class UserDefinedFunction(object): | ||
""" | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -152,7 +152,18 @@ object FunctionRegistry { | |
expression[Substring]("substr"), | ||
expression[Substring]("substring"), | ||
expression[Upper]("ucase"), | ||
expression[Upper]("upper") | ||
expression[Upper]("upper"), | ||
|
||
// datetime functions | ||
expression[DateFormatClass]("dateformat"), | ||
expression[Year]("year"), | ||
expression[Quarter]("quarter"), | ||
expression[Month]("month"), | ||
expression[Day]("day"), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @rxin In Jira you mentioned there should be an alias. Can I just add There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. btw please sort the expressions alphabetically |
||
expression[Hour]("hour"), | ||
expression[Minute]("minute"), | ||
expression[Second]("second"), | ||
expression[WeekOfYear]("weekofyear") | ||
) | ||
|
||
val builtin: FunctionRegistry = { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ignore_unicode_prefix