From d0e2f995ae7a0732c7ea4e6437a246884184fb18 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 01:33:03 -0700 Subject: [PATCH 01/49] date functions --- .../catalyst/analysis/FunctionRegistry.scala | 12 +- .../sql/catalyst/expressions/datetime.scala | 229 ++++++++++++++++++ .../expressions/DateTimeFunctionsSuite.scala | 79 ++++++ .../org/apache/spark/sql/functions.scala | 118 +++++++++ .../spark/sql/DataFrameFunctionsSuite.scala | 153 ++++++++++++ 5 files changed, 590 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5fb3369f85d12..09cbe51c69084 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -152,7 +152,17 @@ object FunctionRegistry { expression[Substring]("substr"), expression[Substring]("substring"), expression[Upper]("ucase"), - expression[Upper]("upper") + expression[Upper]("upper"), + + // datetime functions + expression[DateFormat]("dateformat"), + expression[Year]("year"), + expression[Month]("month"), + expression[Day]("day"), + expression[Hour]("hour"), + expression[Minute]("minute"), + expression[Second]("second"), + expression[WeekOfYear]("weekofyear") ) val builtin: FunctionRegistry = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala new file mode 100644 index 0000000000000..f5675cc4560df --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.sql.Date +import java.text.SimpleDateFormat + +import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +case class DateFormat(left: Expression, right: Expression) + extends BinaryExpression with ExpectsInputTypes { + + override def dataType: DataType = StringType + + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) + + override def foldable: Boolean = left.foldable && right.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + val valueLeft = left.eval(input) + if (valueLeft == null) { + null + } else { + val valueRight = right.eval(input) + if (valueRight == null) { + null + } else { + val sdf = new SimpleDateFormat(valueRight.asInstanceOf[UTF8String].toString) + left.dataType match { + case TimestampType => + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) + case DateType => + UTF8String.fromString(sdf.format(DateTimeUtils.toJavaDate(valueLeft.asInstanceOf[Int]))) + case StringType => + UTF8String.fromString( + sdf.format(DateTimeUtils.stringToTime(valueLeft.asInstanceOf[UTF8String].toString))) + } + } + } + } + + override def toString: String = s"DateFormat($left, $right)" + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val sdf = "java.text.SimpleDateFormat" + val utf8 = "org.apache.spark.unsafe.types.UTF8String" + val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + + val calc = left.dataType match { + case TimestampType => + s"""$utf8.fromString(sdf.format(new java.sql.Date(${eval1.primitive} / 10000)));""" + case DateType => + s"""$utf8.fromString( + sdf.format($dtUtils.toJavaDate(${eval1.primitive})));""" + case StringType => + s""" + $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime()))); + """ + } + + s""" + ${eval1.code} + boolean ${ev.isNull} = ${eval1.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${eval2.code} + if (!${eval2.isNull}) { + $sdf sdf = new $sdf(${eval2.primitive}.toString()); + ${ev.primitive} = $calc + } else { + ${ev.isNull} = true; + } + } + """ + } +} + +case class Year(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("y")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } + +} + +case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("M")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } +} + +case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("d")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } + +} + +case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("H")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } +} + +case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("m")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } +} + +case class Second(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("s")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } +} + +case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormat(child, Literal("w")).eval(input) match { + case null => null + case x: UTF8String => x.toString.toInt + } + } + + +} \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala new file mode 100644 index 0000000000000..e636dd45f6859 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.sql.{Timestamp, Date} +import java.text.SimpleDateFormat + +import org.apache.spark.SparkFunSuite + +class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + test("DateFormat") { + checkEvaluation(DateFormat(Literal(d), Literal("y")), "2015") + checkEvaluation(DateFormat(Literal(d.toString), Literal("y")), "2015") + checkEvaluation(DateFormat(Literal(ts), Literal("y")), "2013") + } + + test("Year") { + checkEvaluation(Year(Literal(d)), 2015) + checkEvaluation(Year(Literal(d.toString)), 2015) + checkEvaluation(Year(Literal(ts)), 2013) + } + + test("Month") { + checkEvaluation(Month(Literal(d)), 4) + checkEvaluation(Month(Literal(d.toString)), 4) + checkEvaluation(Month(Literal(ts)), 4) + } + + test("Day") { + checkEvaluation(Day(Literal(d)), 8) + checkEvaluation(Day(Literal(d.toString)), 8) + checkEvaluation(Day(Literal(ts)), 8) + } + + test("Hour") { + checkEvaluation(Hour(Literal(d)), 0) + checkEvaluation(Hour(Literal(d.toString)), 0) + checkEvaluation(Hour(Literal(ts)), 13) + } + + test("Minute") { + checkEvaluation(Minute(Literal(d)), 0) + checkEvaluation(Minute(Literal(d.toString)), 0) + checkEvaluation(Minute(Literal(ts)), 10) + } + + test("Seconds") { + checkEvaluation(Second(Literal(d)), 0) + checkEvaluation(Second(Literal(d.toString)), 0) + checkEvaluation(Second(Literal(ts)), 15) + } + + test("WeekOfYear") { + checkEvaluation(WeekOfYear(Literal(d)), 15) + checkEvaluation(WeekOfYear(Literal(d.toString)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 15) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 38d9085a505fb..6f9500a8bdec9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -38,6 +38,7 @@ import org.apache.spark.util.Utils * @groupname normal_funcs Non-aggregate functions * @groupname math_funcs Math functions * @groupname misc_funcs Misc functions + * @groupname datetime_funcs Date and time functions * @groupname window_funcs Window functions * @groupname string_funcs String functions * @groupname Ungrouped Support functions for DataFrames. @@ -1431,6 +1432,123 @@ object functions { * @since 1.5.0 */ def strlen(columnName: String): Column = strlen(Column(columnName)) + ////////////////////////////////////////////////////////////////////////////////////////////// + // DateTime functions + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** + * Converts a date/timestamp/string to a value of string in the format specified by the date + * format given by the second argument + * @group datetime_funcs + * @since 1.5.0 + */ + def dateFormat(l: Column, r: Column): Column = DateFormat(l.expr, r.expr) + + /** + * Converts a date/timestamp/string to a value of string in the format specified by the date + * format given by the second argument + * @group datetime_funcs + * @since 1.5.0 + */ + def dateFormat(l: String, r: String): Column = dateFormat(Column(l), Column(r)) + + /** + * Extracts the year as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def year(e: Column): Column = Year(e.expr) + + /** + * Extracts the year as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def year(columnName: String): Column = year(Column(columnName)) + + /** + * Extracts the month as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def month(e: Column): Column = Month(e.expr) + + /** + * Extracts the month as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def month(columnName: String): Column = month(Column(columnName)) + + /** + * Extracts the day as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def day(e: Column): Column = Day(e.expr) + + /** + * Extracts the day as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def day(columnName: String): Column = day(Column(columnName)) + + /** + * Extracts the hours as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def hour(e: Column): Column = Hour(e.expr) + + /** + * Extracts the hours as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def hour(columnName: String): Column = hour(Column(columnName)) + + /** + * Extracts the minutes as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def minute(e: Column): Column = Minute(e.expr) + + /** + * Extracts the minutes as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def minute(columnName: String): Column = minute(Column(columnName)) + + /** + * Extracts the seconds as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def second(e: Column): Column = Second(e.expr) + + /** + * Extracts the seconds as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def second(columnName: String): Column = second(Column(columnName)) + + /** + * Extracts the week number as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def weekOfYear(e: Column): Column = WeekOfYear(e.expr) + + /** + * Extracts the week number as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def weekOfYear(columnName: String): Column = weekOfYear(Column(columnName)) ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 8b53b384a22fd..ea395e1a81085 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.sql.{Timestamp, Date} +import java.text.SimpleDateFormat + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -161,4 +164,154 @@ class DataFrameFunctionsSuite extends QueryTest { Row(l) }) } + + + + test("date format") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), + Row("2015", "2015", "2015")) + } + + test("year") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(year("a"), year("b"), year("c")), + Row(2015, 2015, 2015)) + + checkAnswer( + df.selectExpr("year(a)", "year(b)", "year(c)"), + Row(2015, 2015, 2015)) + } + + test("month") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(month("a"), month("b"), month("c")), + Row(4, 4, 4)) + + checkAnswer( + df.selectExpr("month(a)", "month(b)", "month(c)"), + Row(4, 4, 4)) + } + + test("day") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day("a"), day("b"), day("c")), + Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day(a)", "day(b)", "day(c)"), + Row(8, 8, 8)) + } + + test("hour") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(hour("a"), hour("b"), hour("c")), + Row(0, 0, 13)) + + checkAnswer( + df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + Row(0, 0, 13)) + } + + test("minute") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(minute("a"), minute("b"), minute("c")), + Row(0, 0, 10)) + + checkAnswer( + df.selectExpr("minute(a)", "minute(b)", "minute(c)"), + Row(0, 0, 10)) + } + + test("second") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(hour("a"), hour("b"), hour("c")), + Row(0, 0, 15)) + + checkAnswer( + df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + Row(0, 0, 15)) + } + + test("weekOfYear") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + + val df = Seq(( + d, + d.toString, + ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), + Row(15, 15, 15)) + + checkAnswer( + df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), + Row(15, 15, 15)) + } } From 5ebb2358465abf65b9383bff26947aed06e815b5 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 02:55:33 -0700 Subject: [PATCH 02/49] resolved naming conflict fixed test cases dataframe test issues: - dateformat with date string is null - hours, minutes, seconds is zero for timestamp --- .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../sql/catalyst/expressions/datetime.scala | 16 +++--- .../expressions/DateTimeFunctionsSuite.scala | 6 +-- .../org/apache/spark/sql/functions.scala | 4 +- .../spark/sql/DataFrameFunctionsSuite.scala | 52 ++++++------------- 5 files changed, 30 insertions(+), 50 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 09cbe51c69084..17eed25cd77cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -155,7 +155,7 @@ object FunctionRegistry { expression[Upper]("upper"), // datetime functions - expression[DateFormat]("dateformat"), + expression[DateFormatClass]("dateformat"), expression[Year]("year"), expression[Month]("month"), expression[Day]("day"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index f5675cc4560df..d7f5f5b8b8e38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -case class DateFormat(left: Expression, right: Expression) +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression with ExpectsInputTypes { override def dataType: DataType = StringType @@ -109,7 +109,7 @@ case class Year(child: Expression) extends UnaryExpression with ExpectsInputType override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("y")).eval(input) match { + DateFormatClass(child, Literal("y")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -128,7 +128,7 @@ case class Month(child: Expression) extends UnaryExpression with ExpectsInputTyp override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("M")).eval(input) match { + DateFormatClass(child, Literal("M")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -146,7 +146,7 @@ case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("d")).eval(input) match { + DateFormatClass(child, Literal("d")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -165,7 +165,7 @@ case class Hour(child: Expression) extends UnaryExpression with ExpectsInputType override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("H")).eval(input) match { + DateFormatClass(child, Literal("H")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -183,7 +183,7 @@ case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTy override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("m")).eval(input) match { + DateFormatClass(child, Literal("m")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -201,7 +201,7 @@ case class Second(child: Expression) extends UnaryExpression with ExpectsInputTy override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("s")).eval(input) match { + DateFormatClass(child, Literal("s")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } @@ -219,7 +219,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInp override def nullable: Boolean = true override def eval(input: InternalRow): Any = { - DateFormat(child, Literal("w")).eval(input) match { + DateFormatClass(child, Literal("w")).eval(input) match { case null => null case x: UTF8String => x.toString.toInt } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index e636dd45f6859..cdaccc1ab9740 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -29,9 +29,9 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) test("DateFormat") { - checkEvaluation(DateFormat(Literal(d), Literal("y")), "2015") - checkEvaluation(DateFormat(Literal(d.toString), Literal("y")), "2015") - checkEvaluation(DateFormat(Literal(ts), Literal("y")), "2013") + checkEvaluation(DateFormatClass(Literal(d), Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(d.toString), Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") } test("Year") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 6f9500a8bdec9..1f94942cb117e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1442,7 +1442,7 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def dateFormat(l: Column, r: Column): Column = DateFormat(l.expr, r.expr) + def dateFormat(l: Column, r: String): Column = DateFormatClass(l.expr, Literal(r)) /** * Converts a date/timestamp/string to a value of string in the format specified by the date @@ -1450,7 +1450,7 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ - def dateFormat(l: String, r: String): Column = dateFormat(Column(l), Column(r)) + def dateFormat(l: String, r: String): Column = dateFormat(Column(l), r) /** * Extracts the year as an integer from a given date/timestamp/string diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index ea395e1a81085..e0f4c648e55be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.{Timestamp, Date} +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import org.apache.spark.sql.TestData._ @@ -172,14 +172,15 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2015")) + Row("2015", "2015", "2013")) + + checkAnswer( + df.selectExpr("dateFormat(a, y)", "dateFormat(b, y)", "dateFormat(c, y)"), + Row("2015", "2015", "2013")) } test("year") { @@ -187,18 +188,15 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(year("a"), year("b"), year("c")), - Row(2015, 2015, 2015)) + Row(2015, 2015, 2013)) checkAnswer( df.selectExpr("year(a)", "year(b)", "year(c)"), - Row(2015, 2015, 2015)) + Row(2015, 2015, 2013)) } test("month") { @@ -206,10 +204,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(month("a"), month("b"), month("c")), @@ -225,10 +220,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(day("a"), day("b"), day("c")), @@ -244,10 +236,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(hour("a"), hour("b"), hour("c")), @@ -263,10 +252,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(minute("a"), minute("b"), minute("c")), @@ -282,10 +268,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(hour("a"), hour("b"), hour("c")), @@ -301,10 +284,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq(( - d, - d.toString, - ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), From 4d8049bc79e41622a72e511065645db98708f3c9 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 09:41:17 -0700 Subject: [PATCH 03/49] fixed tests and added type check --- .../sql/catalyst/expressions/datetime.scala | 111 +++++++++++++++--- .../spark/sql/DataFrameFunctionsSuite.scala | 6 +- 2 files changed, 97 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index d7f5f5b8b8e38..46d1c37ffcdfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -20,17 +20,27 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Date import java.text.SimpleDateFormat +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -case class DateFormatClass(left: Expression, right: Expression) - extends BinaryExpression with ExpectsInputTypes { +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression { override def dataType: DataType = StringType - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) + override def checkInputDataTypes(): TypeCheckResult = + (left.dataType, right.dataType) match { + case (null, _) => TypeCheckResult.TypeCheckSuccess + case (_, null) => TypeCheckResult.TypeCheckSuccess + case (_: DateType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case (_: TimestampType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case (_: StringType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"DateFormat accepts date types as first argument, " + + s"and string types as second, not ${left.dataType} and ${right.dataType}") + } override def foldable: Boolean = left.foldable && right.foldable @@ -71,7 +81,7 @@ case class DateFormatClass(left: Expression, right: Expression) val calc = left.dataType match { case TimestampType => - s"""$utf8.fromString(sdf.format(new java.sql.Date(${eval1.primitive} / 10000)));""" + s""""$utf8.fromString(sdf.format(new java.sql.Date(${eval1.primitive} / 10000)));""" case DateType => s"""$utf8.fromString( sdf.format($dtUtils.toJavaDate(${eval1.primitive})));""" @@ -115,6 +125,17 @@ case class Year(child: Expression) extends UnaryExpression with ExpectsInputType } } + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Year accepts date types as argument, " + + s" not ${child.dataType}") + } + } case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -133,6 +154,17 @@ case class Month(child: Expression) extends UnaryExpression with ExpectsInputTyp case x: UTF8String => x.toString.toInt } } + + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Month accepts date types as argument, " + + s" not ${child.dataType}") + } } case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -152,14 +184,23 @@ case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes } } + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Day accepts date types as argument, " + + s" not ${child.dataType}") + } + } -case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Hour(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -170,14 +211,23 @@ case class Hour(child: Expression) extends UnaryExpression with ExpectsInputType case x: UTF8String => x.toString.toInt } } + + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Hour accepts date types as argument, " + + s" not ${child.dataType}") + } } -case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Minute(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -188,14 +238,23 @@ case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTy case x: UTF8String => x.toString.toInt } } + + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Minute accepts date types as argument, " + + s" not ${child.dataType}") + } } -case class Second(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Second(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -206,14 +265,23 @@ case class Second(child: Expression) extends UnaryExpression with ExpectsInputTy case x: UTF8String => x.toString.toInt } } + + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Second accepts date types as argument, " + + s" not ${child.dataType}") + } } -case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class WeekOfYear(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -225,5 +293,14 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInp } } - + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"WeekOfYear accepts date types as argument, " + + s" not ${child.dataType}") + } } \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index e0f4c648e55be..b4bb4987177a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -179,7 +179,7 @@ class DataFrameFunctionsSuite extends QueryTest { Row("2015", "2015", "2013")) checkAnswer( - df.selectExpr("dateFormat(a, y)", "dateFormat(b, y)", "dateFormat(c, y)"), + df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), Row("2015", "2015", "2013")) } @@ -271,11 +271,11 @@ class DataFrameFunctionsSuite extends QueryTest { val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( - df.select(hour("a"), hour("b"), hour("c")), + df.select(second("a"), second("b"), second("c")), Row(0, 0, 15)) checkAnswer( - df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + df.selectExpr("second(a)", "second(b)", "second(c)"), Row(0, 0, 15)) } From 638596f3fd90faa0c0c90d5a0932d5adc3b515d2 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 10:11:23 -0700 Subject: [PATCH 04/49] improved codegen --- .../sql/catalyst/expressions/datetime.scala | 33 +++++++------------ .../spark/sql/DataFrameFunctionsSuite.scala | 8 +++-- 2 files changed, 16 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 46d1c37ffcdfb..9d38f229bad5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -79,18 +79,6 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx val eval1 = left.gen(ctx) val eval2 = right.gen(ctx) - val calc = left.dataType match { - case TimestampType => - s""""$utf8.fromString(sdf.format(new java.sql.Date(${eval1.primitive} / 10000)));""" - case DateType => - s"""$utf8.fromString( - sdf.format($dtUtils.toJavaDate(${eval1.primitive})));""" - case StringType => - s""" - $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime()))); - """ - } - s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; @@ -99,7 +87,14 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx ${eval2.code} if (!${eval2.isNull}) { $sdf sdf = new $sdf(${eval2.primitive}.toString()); - ${ev.primitive} = $calc + Object o = ${eval1.primitive}; + if (o instanceof ${ctx.boxedType(TimestampType)}) { + ${ev.primitive} = $utf8.fromString(sdf.format(new java.sql.Date(Long.parseLong(o.toString()) / 10000))); + } else if (o instanceof Integer) { + ${ev.primitive} = $utf8.fromString(sdf.format($dtUtils.toJavaDate(Integer.parseInt(o.toString())))); + } else { + ${ev.primitive} = $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(o.toString()).getTime()))); + } } else { ${ev.isNull} = true; } @@ -108,12 +103,10 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx } } -case class Year(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Year(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -138,12 +131,10 @@ case class Year(child: Expression) extends UnaryExpression with ExpectsInputType } -case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Month(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true @@ -167,12 +158,10 @@ case class Month(child: Expression) extends UnaryExpression with ExpectsInputTyp } } -case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Day(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType - override def expectedChildTypes: Seq[DataType] = Seq(DateType, StringType, TimestampType) - override def foldable: Boolean = child.foldable override def nullable: Boolean = true diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index b4bb4987177a7..a26e7be9147b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -172,15 +172,17 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts), (ts, d, d.toString), (d.toString, ts, d)).toDF("a", "b", "c") checkAnswer( df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2013")) + Row("2015", "2015", "2013") :: Row("2013", "2015", "2015") :: + Row("2015", "2013", "2015") :: Nil) checkAnswer( df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), - Row("2015", "2015", "2013")) + Row("2015", "2015", "2013") :: Row("2013", "2015", "2015") :: + Row("2015", "2013", "2015") :: Nil) } test("year") { From 849fb4168982d2138c09b530b76d288e3c340df7 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 10:13:35 -0700 Subject: [PATCH 05/49] fixed stupid test --- .../org/apache/spark/sql/DataFrameFunctionsSuite.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index a26e7be9147b8..b4bb4987177a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -172,17 +172,15 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) - val df = Seq((d, d.toString, ts), (ts, d, d.toString), (d.toString, ts, d)).toDF("a", "b", "c") + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") checkAnswer( df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2013") :: Row("2013", "2015", "2015") :: - Row("2015", "2013", "2015") :: Nil) + Row("2015", "2015", "2013")) checkAnswer( df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), - Row("2015", "2015", "2013") :: Row("2013", "2015", "2015") :: - Row("2015", "2013", "2015") :: Nil) + Row("2015", "2015", "2013")) } test("year") { From c7397880b3167ecc2130da8f1c4ff7e413c60877 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 10:41:49 -0700 Subject: [PATCH 06/49] added support for quarter SPARK-8178 --- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../sql/catalyst/expressions/datetime.scala | 28 +++++++++++++++++++ .../expressions/DateTimeFunctionsSuite.scala | 12 ++++++-- .../org/apache/spark/sql/functions.scala | 14 ++++++++++ .../spark/sql/DataFrameFunctionsSuite.scala | 16 +++++++++++ 5 files changed, 68 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 17eed25cd77cc..b1175a3a43c3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -157,6 +157,7 @@ object FunctionRegistry { // datetime functions expression[DateFormatClass]("dateformat"), expression[Year]("year"), + expression[Quarter]("quarter"), expression[Month]("month"), expression[Day]("day"), expression[Hour]("hour"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 9d38f229bad5e..76a66da10b918 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -131,6 +131,34 @@ case class Year(child: Expression) extends UnaryExpression { } +case class Quarter(child: Expression) extends UnaryExpression { + + override def dataType: DataType = IntegerType + + override def foldable: Boolean = child.foldable + + override def nullable: Boolean = true + + override def eval(input: InternalRow): Any = { + DateFormatClass(child, Literal("M")).eval(input) match { + case null => null + case x: UTF8String => (x.toString.toInt - 1) / 3 + 1 + } + } + + override def checkInputDataTypes(): TypeCheckResult = + child.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"Quarter accepts date types as argument, " + + s" not ${child.dataType}") + } + +} + case class Month(child: Expression) extends UnaryExpression { override def dataType: DataType = IntegerType diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index cdaccc1ab9740..4c97b4dad14f7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -26,7 +26,7 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/11/08 13:10:15").getTime) test("DateFormat") { checkEvaluation(DateFormatClass(Literal(d), Literal("y")), "2015") @@ -40,10 +40,16 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Year(Literal(ts)), 2013) } + test("Quarter") { + checkEvaluation(Quarter(Literal(d)), 2) + checkEvaluation(Quarter(Literal(d.toString)), 2) + checkEvaluation(Quarter(Literal(ts)), 4) + } + test("Month") { checkEvaluation(Month(Literal(d)), 4) checkEvaluation(Month(Literal(d.toString)), 4) - checkEvaluation(Month(Literal(ts)), 4) + checkEvaluation(Month(Literal(ts)), 11) } test("Day") { @@ -73,7 +79,7 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal(d)), 15) checkEvaluation(WeekOfYear(Literal(d.toString)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 45) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 1f94942cb117e..0c0bbf4cbc148 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1466,6 +1466,20 @@ object functions { */ def year(columnName: String): Column = year(Column(columnName)) + /** + * Extracts the quarter as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def quarter(e: Column): Column = Quarter(e.expr) + + /** + * Extracts the quarter as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def quarter(columnName: String): Column = quarter(Column(columnName)) + /** * Extracts the month as an integer from a given date/timestamp/string * @group datetime_funcs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index b4bb4987177a7..c1fc73b9e0c94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -199,6 +199,22 @@ class DataFrameFunctionsSuite extends QueryTest { Row(2015, 2015, 2013)) } + test("quarter") { + val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013/11/08 13:10:15").getTime) + + val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(quarter("a"), quarter("b"), quarter("c")), + Row(2, 2, 4)) + + checkAnswer( + df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), + Row(2, 2, 4)) + } + test("month") { val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) From b680db6269f7097f282a6fafe67d33da7ede7f71 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 11:46:05 -0700 Subject: [PATCH 07/49] added codegeneration to all functions --- .../sql/catalyst/expressions/datetime.scala | 238 +++++++++++------- 1 file changed, 145 insertions(+), 93 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 76a66da10b918..a4e5f22aa6c2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -26,37 +26,29 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression { +abstract class DateFormatExpression extends Expression { self: Product => + + protected val date: Expression + + protected val format: Expression - override def dataType: DataType = StringType - - override def checkInputDataTypes(): TypeCheckResult = - (left.dataType, right.dataType) match { - case (null, _) => TypeCheckResult.TypeCheckSuccess - case (_, null) => TypeCheckResult.TypeCheckSuccess - case (_: DateType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case (_: TimestampType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case (_: StringType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"DateFormat accepts date types as first argument, " + - s"and string types as second, not ${left.dataType} and ${right.dataType}") - } - - override def foldable: Boolean = left.foldable && right.foldable + override def foldable: Boolean = date.foldable && format.foldable override def nullable: Boolean = true + override def children: Seq[Expression] = Seq(date, format) + override def eval(input: InternalRow): Any = { - val valueLeft = left.eval(input) + val valueLeft = date.eval(input) if (valueLeft == null) { null } else { - val valueRight = right.eval(input) + val valueRight = format.eval(input) if (valueRight == null) { null } else { val sdf = new SimpleDateFormat(valueRight.asInstanceOf[UTF8String].toString) - left.dataType match { + date.dataType match { case TimestampType => UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) case DateType => @@ -69,15 +61,31 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx } } - override def toString: String = s"DateFormat($left, $right)" + /** + * Called by date format expressions to generate a code block that returns the result + * + * As an example, the following parse the result to int + * {{{ + * defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + * }}} + * + * @param f function that accepts a variable name and returns Java code to parse an + * [[UTF8String]] to the expected output type + */ + + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: String => String): String = { - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = "java.text.SimpleDateFormat" val utf8 = "org.apache.spark.unsafe.types.UTF8String" val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) + val eval1 = date.gen(ctx) + val eval2 = format.gen(ctx) + + val varName = ctx.freshName("resultVar") s""" ${eval1.code} @@ -88,236 +96,280 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx if (!${eval2.isNull}) { $sdf sdf = new $sdf(${eval2.primitive}.toString()); Object o = ${eval1.primitive}; + $utf8 $varName; if (o instanceof ${ctx.boxedType(TimestampType)}) { - ${ev.primitive} = $utf8.fromString(sdf.format(new java.sql.Date(Long.parseLong(o.toString()) / 10000))); - } else if (o instanceof Integer) { - ${ev.primitive} = $utf8.fromString(sdf.format($dtUtils.toJavaDate(Integer.parseInt(o.toString())))); + $varName = $utf8.fromString(sdf.format(new java.sql.Date(Long.parseLong(o.toString()) / 10000))); + } else if (o instanceof ${ctx.boxedType(DateType)}) { + $varName = $utf8.fromString(sdf.format($dtUtils.toJavaDate(Integer.parseInt(o.toString())))); } else { - ${ev.primitive} = $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(o.toString()).getTime()))); + $varName = $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(o.toString()).getTime()))); } + ${ev.primitive} = ${f(varName)}; } else { ${ev.isNull} = true; } } """ } + } -case class Year(child: Expression) extends UnaryExpression { +case class DateFormatClass(date: Expression, format: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override def dataType: DataType = StringType - override def foldable: Boolean = child.foldable + override def checkInputDataTypes(): TypeCheckResult = + (date.dataType, format.dataType) match { + case (null, _) => TypeCheckResult.TypeCheckSuccess + case (_, null) => TypeCheckResult.TypeCheckSuccess + case (_: DateType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case (_: TimestampType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case (_: StringType, _: StringType) => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"DateFormat accepts date types as first argument, " + + s"and string types as second, not ${date.dataType} and ${format.dataType}") + } - override def nullable: Boolean = true + override def toString: String = s"DateFormat($date, $format)" - override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("y")).eval(input) match { - case null => null - case x: UTF8String => x.toString.toInt - } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"$c") } +} + +case class Year(date: Expression) extends DateFormatExpression { + + override protected val format: Expression = Literal("y") + + override def dataType: DataType = IntegerType + override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Year accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } + + override def eval(input: InternalRow): Any = { + super.eval(input) match { + case null => null + case s: UTF8String => s.toString.toInt + } + } } -case class Quarter(child: Expression) extends UnaryExpression { +case class Quarter(date: Expression) extends DateFormatExpression { + + override protected val format: Expression = Literal("M") override def dataType: DataType = IntegerType - override def foldable: Boolean = child.foldable - - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"(Integer.parseInt($c.toString()) - 1) / 3 + 1") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("M")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => (x.toString.toInt - 1) / 3 + 1 + case s: UTF8String => (s.toString.toInt - 1) / 3 + 1 } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Quarter accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class Month(child: Expression) extends UnaryExpression { +case class Month(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("M") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } + override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("M")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Month accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class Day(child: Expression) extends UnaryExpression { +case class Day(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("d") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("d")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Day accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class Hour(child: Expression) extends UnaryExpression { +case class Hour(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("H") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("H")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Hour accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class Minute(child: Expression) extends UnaryExpression { +case class Minute(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("m") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("m")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Minute accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class Second(child: Expression) extends UnaryExpression { +case class Second(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("s") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("s")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"Second accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } -case class WeekOfYear(child: Expression) extends UnaryExpression { +case class WeekOfYear(date: Expression) extends DateFormatExpression { - override def dataType: DataType = IntegerType + override protected val format: Expression = Literal("w") - override def foldable: Boolean = child.foldable + override def dataType: DataType = IntegerType - override def nullable: Boolean = true + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + } override def eval(input: InternalRow): Any = { - DateFormatClass(child, Literal("w")).eval(input) match { + super.eval(input) match { case null => null - case x: UTF8String => x.toString.toInt + case s: UTF8String => s.toString.toInt } } override def checkInputDataTypes(): TypeCheckResult = - child.dataType match { + date.dataType match { case null => TypeCheckResult.TypeCheckSuccess case _: DateType => TypeCheckResult.TypeCheckSuccess case _: TimestampType => TypeCheckResult.TypeCheckSuccess case _: StringType => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"WeekOfYear accepts date types as argument, " + - s" not ${child.dataType}") + s" not ${date.dataType}") } } \ No newline at end of file From a5ea1206b833b3e9680f083fce382842ec1bb485 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Wed, 24 Jun 2015 15:03:01 -0700 Subject: [PATCH 08/49] added python api; changed test to be more meaningful --- python/pyspark/sql/functions.py | 90 +++++++++++++++++++ .../expressions/DateTimeFunctionsSuite.scala | 24 ++--- .../spark/sql/DataFrameFunctionsSuite.scala | 82 ++++++++--------- 3 files changed, 143 insertions(+), 53 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index cfa87aeea193a..ce1bdf8fae24d 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -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): + """ + 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): """ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index 4c97b4dad14f7..840df095d6df3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -24,61 +24,61 @@ import org.apache.spark.SparkFunSuite class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/11/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) test("DateFormat") { checkEvaluation(DateFormatClass(Literal(d), Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(d.toString), Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(sdf.format(d)), Literal("y")), "2015") checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") } test("Year") { checkEvaluation(Year(Literal(d)), 2015) - checkEvaluation(Year(Literal(d.toString)), 2015) + checkEvaluation(Year(Literal(sdf.format(d))), 2015) checkEvaluation(Year(Literal(ts)), 2013) } test("Quarter") { checkEvaluation(Quarter(Literal(d)), 2) - checkEvaluation(Quarter(Literal(d.toString)), 2) + checkEvaluation(Quarter(Literal(sdf.format(d))), 2) checkEvaluation(Quarter(Literal(ts)), 4) } test("Month") { checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Literal(d.toString)), 4) + checkEvaluation(Month(Literal(sdf.format(d))), 4) checkEvaluation(Month(Literal(ts)), 11) } test("Day") { checkEvaluation(Day(Literal(d)), 8) - checkEvaluation(Day(Literal(d.toString)), 8) + checkEvaluation(Day(Literal(sdf.format(d))), 8) checkEvaluation(Day(Literal(ts)), 8) } test("Hour") { checkEvaluation(Hour(Literal(d)), 0) - checkEvaluation(Hour(Literal(d.toString)), 0) + checkEvaluation(Hour(Literal(sdf.format(d))), 13) checkEvaluation(Hour(Literal(ts)), 13) } test("Minute") { checkEvaluation(Minute(Literal(d)), 0) - checkEvaluation(Minute(Literal(d.toString)), 0) + checkEvaluation(Minute(Literal(sdf.format(d))), 10) checkEvaluation(Minute(Literal(ts)), 10) } test("Seconds") { checkEvaluation(Second(Literal(d)), 0) - checkEvaluation(Second(Literal(d.toString)), 0) + checkEvaluation(Second(Literal(sdf.format(d))), 15) checkEvaluation(Second(Literal(ts)), 15) } test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Literal(d.toString)), 15) + checkEvaluation(WeekOfYear(Literal(sdf.format(d))), 15) checkEvaluation(WeekOfYear(Literal(ts)), 45) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index c1fc73b9e0c94..7c042f76b11da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -168,9 +168,9 @@ class DataFrameFunctionsSuite extends QueryTest { test("date format") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") @@ -184,11 +184,11 @@ class DataFrameFunctionsSuite extends QueryTest { } test("year") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(year("a"), year("b"), year("c")), @@ -200,11 +200,11 @@ class DataFrameFunctionsSuite extends QueryTest { } test("quarter") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/11/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(quarter("a"), quarter("b"), quarter("c")), @@ -216,11 +216,11 @@ class DataFrameFunctionsSuite extends QueryTest { } test("month") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(month("a"), month("b"), month("c")), @@ -232,11 +232,11 @@ class DataFrameFunctionsSuite extends QueryTest { } test("day") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(day("a"), day("b"), day("c")), @@ -248,59 +248,59 @@ class DataFrameFunctionsSuite extends QueryTest { } test("hour") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(hour("a"), hour("b"), hour("c")), - Row(0, 0, 13)) + Row(0, 13, 13)) checkAnswer( df.selectExpr("hour(a)", "hour(b)", "hour(c)"), - Row(0, 0, 13)) + Row(0, 13, 13)) } test("minute") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(minute("a"), minute("b"), minute("c")), - Row(0, 0, 10)) + Row(0, 10, 10)) checkAnswer( df.selectExpr("minute(a)", "minute(b)", "minute(c)"), - Row(0, 0, 10)) + Row(0, 10, 10)) } test("second") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(second("a"), second("b"), second("c")), - Row(0, 0, 15)) + Row(0, 15, 15)) checkAnswer( df.selectExpr("second(a)", "second(b)", "second(c)"), - Row(0, 0, 15)) + Row(0, 15, 15)) } test("weekOfYear") { - val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - val d = new Date(sdf.parse("2015/04/08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013/04/08 13:10:15").getTime) + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), From 02efc5da64d3f95024b860e50a4648d989496c9f Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 26 Jun 2015 16:01:49 -0700 Subject: [PATCH 09/49] removed doubled code --- .../sql/catalyst/expressions/datetime.scala | 241 ++++++++---------- .../expressions/DateTimeFunctionsSuite.scala | 11 + 2 files changed, 119 insertions(+), 133 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index a4e5f22aa6c2d..74c8ea4e3a0dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -27,27 +27,28 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String abstract class DateFormatExpression extends Expression { self: Product => + + protected val format: String + + protected val caller: String protected val date: Expression - - protected val format: Expression - override def foldable: Boolean = date.foldable && format.foldable + override def foldable: Boolean = date.foldable override def nullable: Boolean = true - override def children: Seq[Expression] = Seq(date, format) + override def children: Seq[Expression] = Seq(date) override def eval(input: InternalRow): Any = { val valueLeft = date.eval(input) if (valueLeft == null) { null } else { - val valueRight = format.eval(input) - if (valueRight == null) { + if (format == null) { null } else { - val sdf = new SimpleDateFormat(valueRight.asInstanceOf[UTF8String].toString) + val sdf = new SimpleDateFormat(format) date.dataType match { case TimestampType => UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) @@ -55,12 +56,24 @@ abstract class DateFormatExpression extends Expression { self: Product => UTF8String.fromString(sdf.format(DateTimeUtils.toJavaDate(valueLeft.asInstanceOf[Int]))) case StringType => UTF8String.fromString( - sdf.format(DateTimeUtils.stringToTime(valueLeft.asInstanceOf[UTF8String].toString))) + sdf.format(DateTimeUtils.stringToTime(valueLeft.toString))) } } } } + override def checkInputDataTypes(): TypeCheckResult = + date.dataType match { + case null => TypeCheckResult.TypeCheckSuccess + case _: DateType => TypeCheckResult.TypeCheckSuccess + case _: TimestampType => TypeCheckResult.TypeCheckSuccess + case _: StringType => TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure(s"$caller accepts date types as argument, " + + s" not ${date.dataType}") + } + + /** * Called by date format expressions to generate a code block that returns the result * @@ -78,48 +91,39 @@ abstract class DateFormatExpression extends Expression { self: Product => ev: GeneratedExpressionCode, f: String => String): String = { - val sdf = "java.text.SimpleDateFormat" - val utf8 = "org.apache.spark.unsafe.types.UTF8String" + val sdf = classOf[SimpleDateFormat].getName val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" val eval1 = date.gen(ctx) - val eval2 = format.gen(ctx) - val varName = ctx.freshName("resultVar") + val parseInput = date.dataType match { + case StringType => s"new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime())" + case TimestampType => s"new java.sql.Date(${eval1.primitive} / 10000)" + case DateType => s"$dtUtils.toJavaDate(${eval1.primitive})" + } s""" ${eval1.code} boolean ${ev.isNull} = ${eval1.isNull}; ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; if (!${ev.isNull}) { - ${eval2.code} - if (!${eval2.isNull}) { - $sdf sdf = new $sdf(${eval2.primitive}.toString()); - Object o = ${eval1.primitive}; - $utf8 $varName; - if (o instanceof ${ctx.boxedType(TimestampType)}) { - $varName = $utf8.fromString(sdf.format(new java.sql.Date(Long.parseLong(o.toString()) / 10000))); - } else if (o instanceof ${ctx.boxedType(DateType)}) { - $varName = $utf8.fromString(sdf.format($dtUtils.toJavaDate(Integer.parseInt(o.toString())))); - } else { - $varName = $utf8.fromString(sdf.format(new java.sql.Date($dtUtils.stringToTime(o.toString()).getTime()))); - } - ${ev.primitive} = ${f(varName)}; - } else { - ${ev.isNull} = true; - } + $sdf sdf = new $sdf("$format"); + ${ctx.stringType} s = ${ctx.stringType}.fromString(sdf.format($parseInput)); + ${ev.primitive} = ${f("s")}; + } else { + ${ev.isNull} = true; } """ } } -case class DateFormatClass(date: Expression, format: Expression) extends DateFormatExpression { +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression { override def dataType: DataType = StringType override def checkInputDataTypes(): TypeCheckResult = - (date.dataType, format.dataType) match { + (left.dataType, right.dataType) match { case (null, _) => TypeCheckResult.TypeCheckSuccess case (_, null) => TypeCheckResult.TypeCheckSuccess case (_: DateType, _: StringType) => TypeCheckResult.TypeCheckSuccess @@ -127,33 +131,69 @@ case class DateFormatClass(date: Expression, format: Expression) extends DateFor case (_: StringType, _: StringType) => TypeCheckResult.TypeCheckSuccess case _ => TypeCheckResult.TypeCheckFailure(s"DateFormat accepts date types as first argument, " + - s"and string types as second, not ${date.dataType} and ${format.dataType}") + s"and string types as second, not ${left.dataType} and ${right.dataType}") } - override def toString: String = s"DateFormat($date, $format)" + override def toString: String = s"DateFormat($left, $right)" - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") + override def eval(input: InternalRow): Any = { + val valueLeft = left.eval(input) + if (valueLeft == null) { + null + } else { + val valueRight = right.eval(input) + if (valueRight == null) { + null + } else { + val sdf = new SimpleDateFormat(valueRight.toString) + left.dataType match { + case TimestampType => + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) + case DateType => + UTF8String.fromString(sdf.format(DateTimeUtils.toJavaDate(valueLeft.asInstanceOf[Int]))) + case StringType => + UTF8String.fromString( + sdf.format(DateTimeUtils.stringToTime(valueLeft.toString))) + } + } + } } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val sdf = classOf[SimpleDateFormat].getName + val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + + val eval1 = left.gen(ctx) + val eval2 = right.gen(ctx) + + val parseInput = left.dataType match { + case StringType => s"new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime())" + case TimestampType => s"new java.sql.Date(${eval1.primitive} / 10000)" + case DateType => s"$dtUtils.toJavaDate(${eval1.primitive})" + } + + s""" + ${eval1.code} + ${eval2.code} + boolean ${ev.isNull} = ${eval1.isNull} || ${eval2.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $sdf sdf = new $sdf(${eval2.primitive}.toString()); + ${ev.primitive} = ${ctx.stringType}.fromString(sdf.format($parseInput)); + } else { + ${ev.isNull} = true; + } + """ + } } case class Year(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("y") + override protected val format: String = "y" - override def dataType: DataType = IntegerType + override protected val caller: String = "Year" - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Year accepts date types as argument, " + - s" not ${date.dataType}") - } + override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") @@ -168,8 +208,10 @@ case class Year(date: Expression) extends DateFormatExpression { } case class Quarter(date: Expression) extends DateFormatExpression { - - override protected val format: Expression = Literal("M") + + override protected val format: String = "M" + + override protected val caller: String = "Quarter" override def dataType: DataType = IntegerType @@ -183,23 +225,13 @@ case class Quarter(date: Expression) extends DateFormatExpression { case s: UTF8String => (s.toString.toInt - 1) / 3 + 1 } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Quarter accepts date types as argument, " + - s" not ${date.dataType}") - } - } case class Month(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("M") + override protected val format: String = "M" + + override protected val caller: String = "Month" override def dataType: DataType = IntegerType @@ -215,22 +247,13 @@ case class Month(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Month accepts date types as argument, " + - s" not ${date.dataType}") - } } case class Day(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("d") + override protected val format: String = "d" + + override protected val caller: String = "Day" override def dataType: DataType = IntegerType @@ -244,23 +267,13 @@ case class Day(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Day accepts date types as argument, " + - s" not ${date.dataType}") - } - } case class Hour(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("H") + override protected val format: String = "H" + + override protected val caller: String = "Hour" override def dataType: DataType = IntegerType @@ -274,22 +287,13 @@ case class Hour(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Hour accepts date types as argument, " + - s" not ${date.dataType}") - } } case class Minute(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("m") + override protected val format: String = "m" + + override protected val caller: String = "Minute" override def dataType: DataType = IntegerType @@ -303,22 +307,13 @@ case class Minute(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Minute accepts date types as argument, " + - s" not ${date.dataType}") - } } case class Second(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("s") + override protected val format: String = "s" + + override protected val caller: String = "Second" override def dataType: DataType = IntegerType @@ -332,22 +327,13 @@ case class Second(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"Second accepts date types as argument, " + - s" not ${date.dataType}") - } } case class WeekOfYear(date: Expression) extends DateFormatExpression { - override protected val format: Expression = Literal("w") + override protected val format: String = "w" + + override protected val caller: String = "WeekOfYear" override def dataType: DataType = IntegerType @@ -361,15 +347,4 @@ case class WeekOfYear(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } - - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"WeekOfYear accepts date types as argument, " + - s" not ${date.dataType}") - } } \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index 840df095d6df3..941733bf2214d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -21,6 +21,7 @@ import java.sql.{Timestamp, Date} import java.text.SimpleDateFormat import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{StringType, DateType} class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -29,54 +30,64 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) test("DateFormat") { + checkEvaluation(DateFormatClass(Literal.create(null, DateType), Literal("y")), null) + checkEvaluation(DateFormatClass(Literal(d), Literal.create(null, StringType)), null) checkEvaluation(DateFormatClass(Literal(d), Literal("y")), "2015") checkEvaluation(DateFormatClass(Literal(sdf.format(d)), Literal("y")), "2015") checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") } test("Year") { + checkEvaluation(Year(Literal.create(null, DateType)), null) checkEvaluation(Year(Literal(d)), 2015) checkEvaluation(Year(Literal(sdf.format(d))), 2015) checkEvaluation(Year(Literal(ts)), 2013) } test("Quarter") { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Literal(d)), 2) checkEvaluation(Quarter(Literal(sdf.format(d))), 2) checkEvaluation(Quarter(Literal(ts)), 4) } test("Month") { + checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Literal(d)), 4) checkEvaluation(Month(Literal(sdf.format(d))), 4) checkEvaluation(Month(Literal(ts)), 11) } test("Day") { + checkEvaluation(Day(Literal.create(null, DateType)), null) checkEvaluation(Day(Literal(d)), 8) checkEvaluation(Day(Literal(sdf.format(d))), 8) checkEvaluation(Day(Literal(ts)), 8) } test("Hour") { + checkEvaluation(Hour(Literal.create(null, DateType)), null) checkEvaluation(Hour(Literal(d)), 0) checkEvaluation(Hour(Literal(sdf.format(d))), 13) checkEvaluation(Hour(Literal(ts)), 13) } test("Minute") { + checkEvaluation(Minute(Literal.create(null, DateType)), null) checkEvaluation(Minute(Literal(d)), 0) checkEvaluation(Minute(Literal(sdf.format(d))), 10) checkEvaluation(Minute(Literal(ts)), 10) } test("Seconds") { + checkEvaluation(Second(Literal.create(null, DateType)), null) checkEvaluation(Second(Literal(d)), 0) checkEvaluation(Second(Literal(sdf.format(d))), 15) checkEvaluation(Second(Literal(ts)), 15) } test("WeekOfYear") { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) checkEvaluation(WeekOfYear(Literal(d)), 15) checkEvaluation(WeekOfYear(Literal(sdf.format(d))), 15) checkEvaluation(WeekOfYear(Literal(ts)), 45) From 356df788e62dcc38a058a7f56c8a8a6498289efc Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sun, 28 Jun 2015 19:32:41 -0700 Subject: [PATCH 10/49] rely on cast mechanism of Spark. Simplified implementation --- .../sql/catalyst/expressions/datetime.scala | 179 ++++-------------- .../expressions/DateTimeFunctionsSuite.scala | 44 +++-- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- 3 files changed, 65 insertions(+), 160 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 74c8ea4e3a0dd..20db44abf3e8c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -26,22 +26,15 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -abstract class DateFormatExpression extends Expression { self: Product => +abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { + self: Product => protected val format: String - protected val caller: String - - protected val date: Expression - - override def foldable: Boolean = date.foldable - - override def nullable: Boolean = true - - override def children: Seq[Expression] = Seq(date) + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) override def eval(input: InternalRow): Any = { - val valueLeft = date.eval(input) + val valueLeft = child.eval(input) if (valueLeft == null) { null } else { @@ -49,92 +42,32 @@ abstract class DateFormatExpression extends Expression { self: Product => null } else { val sdf = new SimpleDateFormat(format) - date.dataType match { - case TimestampType => - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - case DateType => - UTF8String.fromString(sdf.format(DateTimeUtils.toJavaDate(valueLeft.asInstanceOf[Int]))) - case StringType => - UTF8String.fromString( - sdf.format(DateTimeUtils.stringToTime(valueLeft.toString))) - } + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) } } } - override def checkInputDataTypes(): TypeCheckResult = - date.dataType match { - case null => TypeCheckResult.TypeCheckSuccess - case _: DateType => TypeCheckResult.TypeCheckSuccess - case _: TimestampType => TypeCheckResult.TypeCheckSuccess - case _: StringType => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"$caller accepts date types as argument, " + - s" not ${date.dataType}") - } - - - /** - * Called by date format expressions to generate a code block that returns the result - * - * As an example, the following parse the result to int - * {{{ - * defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - * }}} - * - * @param f function that accepts a variable name and returns Java code to parse an - * [[UTF8String]] to the expected output type - */ - - protected def defineCodeGen( + override protected def defineCodeGen( ctx: CodeGenContext, ev: GeneratedExpressionCode, f: String => String): String = { val sdf = classOf[SimpleDateFormat].getName - val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" - - val eval1 = date.gen(ctx) - - val parseInput = date.dataType match { - case StringType => s"new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime())" - case TimestampType => s"new java.sql.Date(${eval1.primitive} / 10000)" - case DateType => s"$dtUtils.toJavaDate(${eval1.primitive})" - } - - s""" - ${eval1.code} - boolean ${ev.isNull} = ${eval1.isNull}; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - $sdf sdf = new $sdf("$format"); - ${ctx.stringType} s = ${ctx.stringType}.fromString(sdf.format($parseInput)); - ${ev.primitive} = ${f("s")}; - } else { - ${ev.isNull} = true; - } - """ + super.defineCodeGen(ctx, ev, (x) => { + f(s"""${ctx.stringType}.fromString((new $sdf("$format")).format(new java.sql.Date($x / 10000)))""") + }) } } -case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression { +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression + with ExpectsInputTypes { override def dataType: DataType = StringType - override def checkInputDataTypes(): TypeCheckResult = - (left.dataType, right.dataType) match { - case (null, _) => TypeCheckResult.TypeCheckSuccess - case (_, null) => TypeCheckResult.TypeCheckSuccess - case (_: DateType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case (_: TimestampType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case (_: StringType, _: StringType) => TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"DateFormat accepts date types as first argument, " + - s"and string types as second, not ${left.dataType} and ${right.dataType}") - } - override def toString: String = s"DateFormat($left, $right)" + + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) override def eval(input: InternalRow): Any = { val valueLeft = left.eval(input) @@ -146,53 +79,23 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx null } else { val sdf = new SimpleDateFormat(valueRight.toString) - left.dataType match { - case TimestampType => - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - case DateType => - UTF8String.fromString(sdf.format(DateTimeUtils.toJavaDate(valueLeft.asInstanceOf[Int]))) - case StringType => - UTF8String.fromString( - sdf.format(DateTimeUtils.stringToTime(valueLeft.toString))) - } + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) } } } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName - val dtUtils = "org.apache.spark.sql.catalyst.util.DateTimeUtils" - - val eval1 = left.gen(ctx) - val eval2 = right.gen(ctx) - - val parseInput = left.dataType match { - case StringType => s"new java.sql.Date($dtUtils.stringToTime(${eval1.primitive}.toString()).getTime())" - case TimestampType => s"new java.sql.Date(${eval1.primitive} / 10000)" - case DateType => s"$dtUtils.toJavaDate(${eval1.primitive})" - } - - s""" - ${eval1.code} - ${eval2.code} - boolean ${ev.isNull} = ${eval1.isNull} || ${eval2.isNull}; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - $sdf sdf = new $sdf(${eval2.primitive}.toString()); - ${ev.primitive} = ${ctx.stringType}.fromString(sdf.format($parseInput)); - } else { - ${ev.isNull} = true; - } - """ + defineCodeGen(ctx, ev, (x, y) => { + s"""${ctx.stringType}.fromString((new $sdf($y.toString())).format(new java.sql.Date($x / 10000)))""" + }) } } -case class Year(date: Expression) extends DateFormatExpression { +case class Year(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "y" - override protected val caller: String = "Year" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -205,14 +108,14 @@ case class Year(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Year($child)" } -case class Quarter(date: Expression) extends DateFormatExpression { +case class Quarter(child: Expression) extends DateFormatExpression { override protected val format: String = "M" - override protected val caller: String = "Quarter" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -225,18 +128,16 @@ case class Quarter(date: Expression) extends DateFormatExpression { case s: UTF8String => (s.toString.toInt - 1) / 3 + 1 } } + + override def toString: String = s"Quarter($child)" } -case class Month(date: Expression) extends DateFormatExpression { +case class Month(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "M" - override protected val caller: String = "Month" - override def dataType: DataType = IntegerType - override def nullable: Boolean = true - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") } @@ -247,14 +148,14 @@ case class Month(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Month($child)" } -case class Day(date: Expression) extends DateFormatExpression { +case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "d" - override protected val caller: String = "Day" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -267,14 +168,14 @@ case class Day(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Day($child)" } -case class Hour(date: Expression) extends DateFormatExpression { +case class Hour(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "H" - override protected val caller: String = "Hour" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -287,14 +188,14 @@ case class Hour(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Hour($child)" } -case class Minute(date: Expression) extends DateFormatExpression { +case class Minute(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "m" - override protected val caller: String = "Minute" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -307,14 +208,14 @@ case class Minute(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Minute($child)" } -case class Second(date: Expression) extends DateFormatExpression { +case class Second(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "s" - override protected val caller: String = "Second" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -327,14 +228,14 @@ case class Second(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"Second($child)" } -case class WeekOfYear(date: Expression) extends DateFormatExpression { +case class WeekOfYear(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override protected val format: String = "w" - override protected val caller: String = "WeekOfYear" - override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -347,4 +248,6 @@ case class WeekOfYear(date: Expression) extends DateFormatExpression { case s: UTF8String => s.toString.toInt } } + + override def toString: String = s"WeekOfYear($child)" } \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index 941733bf2214d..27d15fcf807ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -21,7 +21,7 @@ import java.sql.{Timestamp, Date} import java.text.SimpleDateFormat import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.types.{StringType, DateType} +import org.apache.spark.sql.types.{TimestampType, StringType, DateType} class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -30,66 +30,68 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) test("DateFormat") { - checkEvaluation(DateFormatClass(Literal.create(null, DateType), Literal("y")), null) - checkEvaluation(DateFormatClass(Literal(d), Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Literal(d), Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(sdf.format(d)), Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") } test("Year") { checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Literal(d)), 2015) - checkEvaluation(Year(Literal(sdf.format(d))), 2015) + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) checkEvaluation(Year(Literal(ts)), 2013) } test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Literal(d)), 2) - checkEvaluation(Quarter(Literal(sdf.format(d))), 2) + checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) checkEvaluation(Quarter(Literal(ts)), 4) } test("Month") { checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Literal(sdf.format(d))), 4) + checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) + checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) checkEvaluation(Month(Literal(ts)), 11) } test("Day") { checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Literal(d)), 8) - checkEvaluation(Day(Literal(sdf.format(d))), 8) + checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) + checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) checkEvaluation(Day(Literal(ts)), 8) } test("Hour") { checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Literal(d)), 0) - checkEvaluation(Hour(Literal(sdf.format(d))), 13) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) checkEvaluation(Hour(Literal(ts)), 13) } test("Minute") { checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Literal(d)), 0) - checkEvaluation(Minute(Literal(sdf.format(d))), 10) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) checkEvaluation(Minute(Literal(ts)), 10) } test("Seconds") { checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Literal(d)), 0) - checkEvaluation(Second(Literal(sdf.format(d))), 15) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) checkEvaluation(Second(Literal(ts)), 15) } test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Literal(sdf.format(d))), 15) + checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) checkEvaluation(WeekOfYear(Literal(ts)), 45) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 7c042f76b11da..33b2e93510699 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -172,7 +172,7 @@ class DataFrameFunctionsSuite extends QueryTest { val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, d.toString, ts)).toDF("a", "b", "c") + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), From 3bfac90e8ea986c16a871bfffa5d4fa58e9dd774 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sun, 28 Jun 2015 21:29:46 -0700 Subject: [PATCH 11/49] fixed style --- .../spark/sql/catalyst/expressions/datetime.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 20db44abf3e8c..c547079721de6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -54,10 +54,11 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp val sdf = classOf[SimpleDateFormat].getName super.defineCodeGen(ctx, ev, (x) => { - f(s"""${ctx.stringType}.fromString((new $sdf("$format")).format(new java.sql.Date($x / 10000)))""") + f(s"""${ctx.stringType}.fromString((new $sdf("$format")) + .format(new java.sql.Date($x / 10000)))""") }) } - + } case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression @@ -66,7 +67,7 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def dataType: DataType = StringType override def toString: String = s"DateFormat($left, $right)" - + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) override def eval(input: InternalRow): Any = { @@ -81,13 +82,14 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx val sdf = new SimpleDateFormat(valueRight.toString) UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) } - } + } } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName defineCodeGen(ctx, ev, (x, y) => { - s"""${ctx.stringType}.fromString((new $sdf($y.toString())).format(new java.sql.Date($x / 10000)))""" + s"""${ctx.stringType}.fromString((new $sdf($y.toString())) + .format(new java.sql.Date($x / 10000)))""" }) } } @@ -250,4 +252,4 @@ case class WeekOfYear(child: Expression) extends DateFormatExpression with Expec } override def toString: String = s"WeekOfYear($child)" -} \ No newline at end of file +} From 5fe74e178097ac4c98ebc4ca50d842f00bbe157e Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sun, 28 Jun 2015 21:51:01 -0700 Subject: [PATCH 12/49] fixed python style --- python/pyspark/sql/functions.py | 36 ++++++++++++++++++++++++--------- 1 file changed, 27 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index ce1bdf8fae24d..04229b07c3b9c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -486,46 +486,56 @@ def ntile(n): sc = SparkContext._active_spark_context return Column(sc._jvm.functions.ntile(int(n))) + @since(1.5) def dateFormat(dateCol, formatCol): """ - 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() + Convert the given date into the format specified by the second argument. + Return type is always string. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.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): """ @@ -536,41 +546,49 @@ def day(col): 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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df0.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> dfo.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df0.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() + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(weekOfYear('a').alias('week')).collect() [Row(week=15)] """ sc = SparkContext._active_spark_context From a8edebde1d62499ea35c5e0e71b7bc92526a369f Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Mon, 29 Jun 2015 16:37:44 -0700 Subject: [PATCH 13/49] use Calendar instead of SimpleDateFormat --- python/pyspark/sql/functions.py | 8 ++ .../sql/catalyst/expressions/datetime.scala | 110 +++++++----------- 2 files changed, 49 insertions(+), 69 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 04229b07c3b9c..d1749ab1e28d8 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -493,6 +493,7 @@ def dateFormat(dateCol, formatCol): Convert the given date into the format specified by the second argument. Return type is always string. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(dateFormat('a', 'MM/dd/yyy').alias('date')).collect() [Row(date=u'04/08/2015')] """ @@ -505,6 +506,7 @@ def year(col): """ Extract the year of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(year('a').alias('year')).collect() [Row(year=2015)] """ @@ -517,6 +519,7 @@ def quarter(col): """ Extract the quarter of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(quarter('a').alias('quarter')).collect() [Row(quarter=2)] """ @@ -529,6 +532,7 @@ def month(col): """ Extract the month of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(month('a').alias('month')).collect() [Row(month=4)] """ @@ -552,6 +556,7 @@ def hour(col): """ Extract the hours of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df0.select(hour('a').alias('hour')).collect() [Row(hour=13)] """ @@ -564,6 +569,7 @@ def minute(col): """ Extract the minutes of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> dfo.select(minute('a').alias('minute')).collect() [Row(minute=8)] """ @@ -576,6 +582,7 @@ def second(col): """ Extract the seconds of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df0.select(second('a').alias('second')).collect() [Row(second=15)] """ @@ -588,6 +595,7 @@ def weekOfYear(col): """ Extract the week number of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(weekOfYear('a').alias('week')).collect() [Row(week=15)] """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index c547079721de6..39c2ba7fa5bd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Date import java.text.SimpleDateFormat +import java.util.{Calendar, TimeZone, Locale} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} @@ -29,7 +30,9 @@ import org.apache.spark.unsafe.types.UTF8String abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => - protected val format: String + private[this] val offset: Int = TimeZone.getDefault.getRawOffset + + protected val format: Int override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) @@ -38,12 +41,10 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp if (valueLeft == null) { null } else { - if (format == null) { - null - } else { - val sdf = new SimpleDateFormat(format) - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - } + val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(format) } } @@ -52,11 +53,20 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp ev: GeneratedExpressionCode, f: String => String): String = { - val sdf = classOf[SimpleDateFormat].getName - super.defineCodeGen(ctx, ev, (x) => { - f(s"""${ctx.stringType}.fromString((new $sdf("$format")) - .format(new java.sql.Date($x / 10000)))""") - }) + val cal = classOf[Calendar].getName + val cVar = ctx.freshName("cal") + + val eval = child.gen(ctx) + // reuse the previous isNull + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $cal $cVar = $cal.getInstance(); + $cVar.setTimeInMillis(${eval.primitive} / 10000); + ${ev.primitive} = ${f(s"""$cVar.get($format)""")}; + } + """ } } @@ -96,19 +106,16 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx case class Year(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "y" + override protected val format: Int = Calendar.YEAR override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + defineCodeGen(ctx, ev, c => s"$c") } override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + super.eval(input) } override def toString: String = s"Year($child)" @@ -116,18 +123,18 @@ case class Year(child: Expression) extends DateFormatExpression with ExpectsInpu case class Quarter(child: Expression) extends DateFormatExpression { - override protected val format: String = "M" + override protected val format: Int = Calendar.MONTH override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"(Integer.parseInt($c.toString()) - 1) / 3 + 1") + defineCodeGen(ctx, ev, c => s"$c / 3 + 1") } override def eval(input: InternalRow): Any = { super.eval(input) match { case null => null - case s: UTF8String => (s.toString.toInt - 1) / 3 + 1 + case i: Int => i / 3 + 1 } } @@ -136,18 +143,18 @@ case class Quarter(child: Expression) extends DateFormatExpression { case class Month(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "M" + override protected val format: Int = Calendar.MONTH override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") + defineCodeGen(ctx, ev, c => s"($c + 1)") } override def eval(input: InternalRow): Any = { super.eval(input) match { case null => null - case s: UTF8String => s.toString.toInt + case i: Int => i + 1 } } @@ -156,19 +163,12 @@ case class Month(child: Expression) extends DateFormatExpression with ExpectsInp case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "d" + override protected val format: Int = Calendar.DAY_OF_MONTH override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - } - - override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + defineCodeGen(ctx, ev, c => s"$c") } override def toString: String = s"Day($child)" @@ -176,19 +176,12 @@ case class Day(child: Expression) extends DateFormatExpression with ExpectsInput case class Hour(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "H" + override protected val format: Int = Calendar.HOUR_OF_DAY override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - } - - override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + defineCodeGen(ctx, ev, c => s"$c") } override def toString: String = s"Hour($child)" @@ -196,19 +189,12 @@ case class Hour(child: Expression) extends DateFormatExpression with ExpectsInpu case class Minute(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "m" + override protected val format: Int = Calendar.MINUTE override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - } - - override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + defineCodeGen(ctx, ev, c => s"$c") } override def toString: String = s"Minute($child)" @@ -216,19 +202,12 @@ case class Minute(child: Expression) extends DateFormatExpression with ExpectsIn case class Second(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "s" + override protected val format: Int = Calendar.SECOND override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - } - - override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + defineCodeGen(ctx, ev, c => s"$c") } override def toString: String = s"Second($child)" @@ -236,19 +215,12 @@ case class Second(child: Expression) extends DateFormatExpression with ExpectsIn case class WeekOfYear(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: String = "w" + override protected val format: Int = Calendar.WEEK_OF_YEAR override def dataType: DataType = IntegerType override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"Integer.parseInt($c.toString())") - } - - override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case s: UTF8String => s.toString.toInt - } + defineCodeGen(ctx, ev, c => s"$c") } override def toString: String = s"WeekOfYear($child)" From f1204154aaa96d93fd9ef723a91b7a91ba1d3773 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Mon, 29 Jun 2015 21:24:28 -0700 Subject: [PATCH 14/49] improved runtime --- .../sql/catalyst/expressions/datetime.scala | 374 +++++++++++++----- .../expressions/DateTimeFunctionsSuite.scala | 64 +++ 2 files changed, 331 insertions(+), 107 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala index 39c2ba7fa5bd8..f916c37524574 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala @@ -19,21 +19,18 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Date import java.text.SimpleDateFormat -import java.util.{Calendar, TimeZone, Locale} - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import java.util.{Calendar, TimeZone} import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { +abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => - private[this] val offset: Int = TimeZone.getDefault.getRawOffset - protected val format: Int + protected val cntPerInterval: Int + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) override def eval(input: InternalRow): Any = { @@ -41,81 +38,135 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp if (valueLeft == null) { null } else { - val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 - val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(format) + val time = valueLeft.asInstanceOf[Long] / 10000 + val utcTime: Long = time + TimeZone.getDefault.getOffset(time) + ((utcTime / format) % cntPerInterval).toInt } } - override protected def defineCodeGen( + override def genCode( ctx: CodeGenContext, - ev: GeneratedExpressionCode, - f: String => String): String = { + ev: GeneratedExpressionCode): String = { - val cal = classOf[Calendar].getName - val cVar = ctx.freshName("cal") + val tz = classOf[TimeZone].getName + + defineCodeGen(ctx, ev, (c) => + s"""(${ctx.javaType(dataType)}) + ((($c / 10000) + $tz.getDefault().getOffset($c / 10000)) / $format % $cntPerInterval)""" + ) + } +} + +case class Hour(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 * 3600 + + override protected val cntPerInterval: Int = 24 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Hour($child)" +} + +case class Minute(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 * 60 + + override protected val cntPerInterval: Int = 60 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Minute($child)" +} + +case class Second(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 + + override protected val cntPerInterval: Int = 60 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Second($child)" +} + +abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { + self: Product => + + override def dataType: DataType = IntegerType + + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) + + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: (String, String) => String): String = { + + val tz = classOf[TimeZone].getName + + val utcTime = ctx.freshName("utcTime") + val dayInYear = ctx.freshName("dayInYear") + val days = ctx.freshName("days") + val year = ctx.freshName("year") val eval = child.gen(ctx) - // reuse the previous isNull ev.isNull = eval.isNull eval.code + s""" ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; if (!${ev.isNull}) { - $cal $cVar = $cal.getInstance(); - $cVar.setTimeInMillis(${eval.primitive} / 10000); - ${ev.primitive} = ${f(s"""$cVar.get($format)""")}; + long $utcTime = ${eval.primitive} / 10000; + long $days = $utcTime / 1000 / 3600 / 24; + int $year = (int) ($days / 365.24); + int $dayInYear = (int) ($days - $year * 365.24); + ${f(dayInYear, utcTime)} } """ } -} - -case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression - with ExpectsInputTypes { - - override def dataType: DataType = StringType - - override def toString: String = s"DateFormat($left, $right)" - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) - - override def eval(input: InternalRow): Any = { - val valueLeft = left.eval(input) + def eval(input: InternalRow, f: (Int, Long) => Int): Any = { + val valueLeft = child.eval(input) if (valueLeft == null) { null } else { - val valueRight = right.eval(input) - if (valueRight == null) { - null - } else { - val sdf = new SimpleDateFormat(valueRight.toString) - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - } + val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 + val days = utcTime / 1000 / 3600 / 24 + val year = days / 365.24 + val dayInYear = days - year.toInt * 365.24 + f(dayInYear.toInt, utcTime) } } - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val sdf = classOf[SimpleDateFormat].getName - defineCodeGen(ctx, ev, (x, y) => { - s"""${ctx.stringType}.fromString((new $sdf($y.toString())) - .format(new java.sql.Date($x / 10000)))""" - }) - } + override def toString: String = s"Year($child)" } -case class Year(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - override protected val format: Int = Calendar.YEAR - - override def dataType: DataType = IntegerType +case class Year(child: Expression) extends DateFormatExpression { override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 360) { + ${ev.primitive} = (int) (1970 + ($utc / 1000 / 3600 / 24 / 365.24)); + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.YEAR); + } + """) } override def eval(input: InternalRow): Any = { - super.eval(input) + eval(input, (dayInYear, utcTime) => + if (dayInYear > 1 && dayInYear < 360) { + 1970 + (utcTime / 1000 / 3600 / 24 / 365.24).toInt + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.YEAR) + } + ) } override def toString: String = s"Year($child)" @@ -123,105 +174,214 @@ case class Year(child: Expression) extends DateFormatExpression with ExpectsInpu case class Quarter(child: Expression) extends DateFormatExpression { - override protected val format: Int = Calendar.MONTH - - override def dataType: DataType = IntegerType - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c / 3 + 1") + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 90) { + ${ev.primitive} = 1; + } else if ($day > 92 && $day < 181) { + ${ev.primitive} = 2; + } else if ($day > 183 && $day < 273) { + ${ev.primitive} = 3; + } else if ($day > 275 && $day < 364) { + ${ev.primitive} = 4; + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.MONTH) / 3 + 1; + } + """) } override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case i: Int => i / 3 + 1 - } + eval(input, (dayInYear, utcTime) => + dayInYear match { + case i: Int if i > 1 && i < 89 => 1 + case i: Int if i > 93 && i < 180 => 2 + case i: Int if i > 184 && i < 272 => 3 + case i: Int if i > 276 && i < 362 => 4 + case _ => + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.MONTH) / 3 + 1 + } + ) } override def toString: String = s"Quarter($child)" } -case class Month(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - - override protected val format: Int = Calendar.MONTH - - override def dataType: DataType = IntegerType +case class Month(child: Expression) extends DateFormatExpression { override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"($c + 1)") + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 30) { + ${ev.primitive} = 1; + } else if($day > 33 && $day < 58) { + ${ev.primitive} = 2; + } else if($day > 62 && $day < 89) { + ${ev.primitive} = 3; + } else if($day > 93 && $day < 119) { + ${ev.primitive} = 4; + } else if($day > 123 && $day < 150) { + ${ev.primitive} = 5; + } else if($day > 154 && $day < 180) { + ${ev.primitive} = 6; + } else if($day > 184 && $day < 211) { + ${ev.primitive} = 7; + } else if($day > 215 && $day < 242) { + ${ev.primitive} = 8; + } else if($day > 246 && $day < 272) { + ${ev.primitive} = 9; + } else if($day > 276 && $day < 303) { + ${ev.primitive} = 10; + } else if($day > 307 && $day < 333) { + ${ev.primitive} = 11; + } else if($day > 337 && $day < 362) { + ${ev.primitive} = 12; + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.MONTH) + 1; + } + """) } override def eval(input: InternalRow): Any = { - super.eval(input) match { - case null => null - case i: Int => i + 1 - } + eval(input, (dayInYear, utcTime) => + dayInYear match { + case i: Int if i > 1 && i < 30 => 1 + case i: Int if i > 33 && i < 58 => 2 + case i: Int if i > 62 && i < 89 => 3 + case i: Int if i > 93 && i < 119 => 4 + case i: Int if i > 123 && i < 150 => 5 + case i: Int if i > 154 && i < 180 => 6 + case i: Int if i > 184 && i < 211 => 7 + case i: Int if i > 215 && i < 242 => 8 + case i: Int if i > 246 && i < 272 => 9 + case i: Int if i > 276 && i < 303 => 10 + case i: Int if i > 307 && i < 333 => 11 + case i: Int if i > 337 && i < 362 => 12 + case _ => + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.MONTH) + 1 + } + ) } override def toString: String = s"Month($child)" } -case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - - override protected val format: Int = Calendar.DAY_OF_MONTH +case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") + val eval = child.gen(ctx) + + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $cal $c = $cal.getInstance(); + $c.setTimeInMillis(${eval.primitive} / 10000); + ${ev.primitive} = $c.get($cal.DAY_OF_MONTH); + } + """ + } + + override def eval(input: InternalRow): Any = { + val value = child.eval(input) + if (value == null) { + null + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) + c.get(Calendar.DAY_OF_MONTH) + } } override def toString: String = s"Day($child)" } -case class Hour(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - - override protected val format: Int = Calendar.HOUR_OF_DAY +case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") - } - - override def toString: String = s"Hour($child)" -} + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) -case class Minute(child: Expression) extends DateFormatExpression with ExpectsInputTypes { + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val eval = child.gen(ctx) - override protected val format: Int = Calendar.MINUTE + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") - override def dataType: DataType = IntegerType + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $cal $c = $cal.getInstance(); + $c.setTimeInMillis(${eval.primitive} / 10000); + ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); + } + """ + } - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") + override def eval(input: InternalRow): Any = { + val value = child.eval(input) + if (value == null) { + null + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) + c.get(Calendar.WEEK_OF_YEAR) + } } - override def toString: String = s"Minute($child)" + override def toString: String = s"WeekOfYear($child)" } -case class Second(child: Expression) extends DateFormatExpression with ExpectsInputTypes { - - override protected val format: Int = Calendar.SECOND - override def dataType: DataType = IntegerType - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") - } +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression + with ExpectsInputTypes { - override def toString: String = s"Second($child)" -} + override def dataType: DataType = StringType -case class WeekOfYear(child: Expression) extends DateFormatExpression with ExpectsInputTypes { + override def toString: String = s"DateFormat($left, $right)" - override protected val format: Int = Calendar.WEEK_OF_YEAR + override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) - override def dataType: DataType = IntegerType + override def eval(input: InternalRow): Any = { + val valueLeft = left.eval(input) + if (valueLeft == null) { + null + } else { + val valueRight = right.eval(input) + if (valueRight == null) { + null + } else { + val sdf = new SimpleDateFormat(valueRight.toString) + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) + } + } + } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - defineCodeGen(ctx, ev, c => s"$c") + val sdf = classOf[SimpleDateFormat].getName + defineCodeGen(ctx, ev, (x, y) => { + s"""${ctx.stringType}.fromString((new $sdf($y.toString())) + .format(new java.sql.Date($x / 10000)))""" + }) } - - override def toString: String = s"WeekOfYear($child)" } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala index 27d15fcf807ca..c819ed62e6562 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Timestamp, Date} import java.text.SimpleDateFormat +import java.util.{TimeZone, Calendar} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types.{TimestampType, StringType, DateType} @@ -44,6 +45,18 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) checkEvaluation(Year(Literal(ts)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.YEAR)) + } + } + } } test("Quarter") { @@ -51,6 +64,18 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) checkEvaluation(Quarter(Literal(ts)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) / 3 + 1) + } + } + } } test("Month") { @@ -58,6 +83,18 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) checkEvaluation(Month(Literal(ts)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) + 1) + } + } + } } test("Day") { @@ -72,6 +109,17 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) checkEvaluation(Hour(Literal(ts)), 13) + + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) + } + } + } } test("Minute") { @@ -79,6 +127,15 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) checkEvaluation(Minute(Literal(ts)), 10) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) + } + } } test("Seconds") { @@ -86,6 +143,13 @@ class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) + } } test("WeekOfYear") { From 5a105d90cf7e8535593add6a52372fe48b2c633d Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 4 Jul 2015 01:13:01 -0700 Subject: [PATCH 15/49] [SPARK-8199] rebase after #6985 got merged --- .../sql/catalyst/expressions/datetime.scala | 387 ------------------ .../expressions/datetimeFunctions.scala | 367 +++++++++++++++++ .../expressions/DatetimeFunctionsSuite.scala | 138 +++++++ .../apache/spark/sql/DataFrameDateSuite.scala | 150 +++++++ .../spark/sql/DataFrameFunctionsSuite.scala | 146 ------- 5 files changed, 655 insertions(+), 533 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala deleted file mode 100644 index f916c37524574..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetime.scala +++ /dev/null @@ -1,387 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import java.sql.Date -import java.text.SimpleDateFormat -import java.util.{Calendar, TimeZone} -import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTypes { - self: Product => - - protected val format: Int - - protected val cntPerInterval: Int - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) - - override def eval(input: InternalRow): Any = { - val valueLeft = child.eval(input) - if (valueLeft == null) { - null - } else { - val time = valueLeft.asInstanceOf[Long] / 10000 - val utcTime: Long = time + TimeZone.getDefault.getOffset(time) - ((utcTime / format) % cntPerInterval).toInt - } - } - - override def genCode( - ctx: CodeGenContext, - ev: GeneratedExpressionCode): String = { - - val tz = classOf[TimeZone].getName - - defineCodeGen(ctx, ev, (c) => - s"""(${ctx.javaType(dataType)}) - ((($c / 10000) + $tz.getDefault().getOffset($c / 10000)) / $format % $cntPerInterval)""" - ) - } -} - -case class Hour(child: Expression) extends TimeFormatExpression { - - override protected val format: Int = 1000 * 3600 - - override protected val cntPerInterval: Int = 24 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Hour($child)" -} - -case class Minute(child: Expression) extends TimeFormatExpression { - - override protected val format: Int = 1000 * 60 - - override protected val cntPerInterval: Int = 60 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Minute($child)" -} - -case class Second(child: Expression) extends TimeFormatExpression { - - override protected val format: Int = 1000 - - override protected val cntPerInterval: Int = 60 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Second($child)" -} - -abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { - self: Product => - - override def dataType: DataType = IntegerType - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) - - protected def defineCodeGen( - ctx: CodeGenContext, - ev: GeneratedExpressionCode, - f: (String, String) => String): String = { - - val tz = classOf[TimeZone].getName - - val utcTime = ctx.freshName("utcTime") - val dayInYear = ctx.freshName("dayInYear") - val days = ctx.freshName("days") - val year = ctx.freshName("year") - - val eval = child.gen(ctx) - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - long $utcTime = ${eval.primitive} / 10000; - long $days = $utcTime / 1000 / 3600 / 24; - int $year = (int) ($days / 365.24); - int $dayInYear = (int) ($days - $year * 365.24); - ${f(dayInYear, utcTime)} - } - """ - } - - def eval(input: InternalRow, f: (Int, Long) => Int): Any = { - val valueLeft = child.eval(input) - if (valueLeft == null) { - null - } else { - val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 - val days = utcTime / 1000 / 3600 / 24 - val year = days / 365.24 - val dayInYear = days - year.toInt * 365.24 - f(dayInYear.toInt, utcTime) - } - } - - override def toString: String = s"Year($child)" -} - - -case class Year(child: Expression) extends DateFormatExpression { - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => - s""" - if ($day > 1 && $day < 360) { - ${ev.primitive} = (int) (1970 + ($utc / 1000 / 3600 / 24 / 365.24)); - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); - ${ev.primitive} = c.get($cal.YEAR); - } - """) - } - - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => - if (dayInYear > 1 && dayInYear < 360) { - 1970 + (utcTime / 1000 / 3600 / 24 / 365.24).toInt - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(Calendar.YEAR) - } - ) - } - - override def toString: String = s"Year($child)" -} - -case class Quarter(child: Expression) extends DateFormatExpression { - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => - s""" - if ($day > 1 && $day < 90) { - ${ev.primitive} = 1; - } else if ($day > 92 && $day < 181) { - ${ev.primitive} = 2; - } else if ($day > 183 && $day < 273) { - ${ev.primitive} = 3; - } else if ($day > 275 && $day < 364) { - ${ev.primitive} = 4; - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); - ${ev.primitive} = c.get($cal.MONTH) / 3 + 1; - } - """) - } - - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => - dayInYear match { - case i: Int if i > 1 && i < 89 => 1 - case i: Int if i > 93 && i < 180 => 2 - case i: Int if i > 184 && i < 272 => 3 - case i: Int if i > 276 && i < 362 => 4 - case _ => - val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(Calendar.MONTH) / 3 + 1 - } - ) - } - - override def toString: String = s"Quarter($child)" -} - -case class Month(child: Expression) extends DateFormatExpression { - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => - s""" - if ($day > 1 && $day < 30) { - ${ev.primitive} = 1; - } else if($day > 33 && $day < 58) { - ${ev.primitive} = 2; - } else if($day > 62 && $day < 89) { - ${ev.primitive} = 3; - } else if($day > 93 && $day < 119) { - ${ev.primitive} = 4; - } else if($day > 123 && $day < 150) { - ${ev.primitive} = 5; - } else if($day > 154 && $day < 180) { - ${ev.primitive} = 6; - } else if($day > 184 && $day < 211) { - ${ev.primitive} = 7; - } else if($day > 215 && $day < 242) { - ${ev.primitive} = 8; - } else if($day > 246 && $day < 272) { - ${ev.primitive} = 9; - } else if($day > 276 && $day < 303) { - ${ev.primitive} = 10; - } else if($day > 307 && $day < 333) { - ${ev.primitive} = 11; - } else if($day > 337 && $day < 362) { - ${ev.primitive} = 12; - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); - ${ev.primitive} = c.get($cal.MONTH) + 1; - } - """) - } - - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => - dayInYear match { - case i: Int if i > 1 && i < 30 => 1 - case i: Int if i > 33 && i < 58 => 2 - case i: Int if i > 62 && i < 89 => 3 - case i: Int if i > 93 && i < 119 => 4 - case i: Int if i > 123 && i < 150 => 5 - case i: Int if i > 154 && i < 180 => 6 - case i: Int if i > 184 && i < 211 => 7 - case i: Int if i > 215 && i < 242 => 8 - case i: Int if i > 246 && i < 272 => 9 - case i: Int if i > 276 && i < 303 => 10 - case i: Int if i > 307 && i < 333 => 11 - case i: Int if i > 337 && i < 362 => 12 - case _ => - val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(Calendar.MONTH) + 1 - } - ) - } - - override def toString: String = s"Month($child)" -} - -case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { - - override def dataType: DataType = IntegerType - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val eval = child.gen(ctx) - - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - $cal $c = $cal.getInstance(); - $c.setTimeInMillis(${eval.primitive} / 10000); - ${ev.primitive} = $c.get($cal.DAY_OF_MONTH); - } - """ - } - - override def eval(input: InternalRow): Any = { - val value = child.eval(input) - if (value == null) { - null - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) - c.get(Calendar.DAY_OF_MONTH) - } - } - - override def toString: String = s"Day($child)" -} - -case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { - - override def dataType: DataType = IntegerType - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType) - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val eval = child.gen(ctx) - - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - $cal $c = $cal.getInstance(); - $c.setTimeInMillis(${eval.primitive} / 10000); - ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); - } - """ - } - - override def eval(input: InternalRow): Any = { - val value = child.eval(input) - if (value == null) { - null - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) - c.get(Calendar.WEEK_OF_YEAR) - } - } - - override def toString: String = s"WeekOfYear($child)" -} - - -case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression - with ExpectsInputTypes { - - override def dataType: DataType = StringType - - override def toString: String = s"DateFormat($left, $right)" - - override def expectedChildTypes: Seq[DataType] = Seq(TimestampType, StringType) - - override def eval(input: InternalRow): Any = { - val valueLeft = left.eval(input) - if (valueLeft == null) { - null - } else { - val valueRight = right.eval(input) - if (valueRight == null) { - null - } else { - val sdf = new SimpleDateFormat(valueRight.toString) - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - } - } - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val sdf = classOf[SimpleDateFormat].getName - defineCodeGen(ctx, ev, (x, y) => { - s"""${ctx.stringType}.fromString((new $sdf($y.toString())) - .format(new java.sql.Date($x / 10000)))""" - }) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 13ba2f2e5d62d..a97eab5ca2b7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.Date +import java.text.SimpleDateFormat +import java.util.{Calendar, TimeZone} + import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String /** * Returns the current date at the start of query evaluation. @@ -50,3 +55,365 @@ case class CurrentTimestamp() extends LeafExpression { System.currentTimeMillis() * 10000L } } + +abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTypes { + self: Product => + + protected val format: Int + + protected val cntPerInterval: Int + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + override def eval(input: InternalRow): Any = { + val valueLeft = child.eval(input) + if (valueLeft == null) { + null + } else { + val time = valueLeft.asInstanceOf[Long] / 10000 + val utcTime: Long = time + TimeZone.getDefault.getOffset(time) + ((utcTime / format) % cntPerInterval).toInt + } + } + + override def genCode( + ctx: CodeGenContext, + ev: GeneratedExpressionCode): String = { + + val tz = classOf[TimeZone].getName + + defineCodeGen(ctx, ev, (c) => + s"""(${ctx.javaType(dataType)}) + ((($c / 10000) + $tz.getDefault().getOffset($c / 10000)) / $format % $cntPerInterval)""" + ) + } +} + +case class Hour(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 * 3600 + + override protected val cntPerInterval: Int = 24 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Hour($child)" +} + +case class Minute(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 * 60 + + override protected val cntPerInterval: Int = 60 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Minute($child)" +} + +case class Second(child: Expression) extends TimeFormatExpression { + + override protected val format: Int = 1000 + + override protected val cntPerInterval: Int = 60 + + override def dataType: DataType = IntegerType + + override def toString: String = s"Second($child)" +} + +abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { + self: Product => + + override def dataType: DataType = IntegerType + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: (String, String) => String): String = { + + val tz = classOf[TimeZone].getName + + val utcTime = ctx.freshName("utcTime") + val dayInYear = ctx.freshName("dayInYear") + val days = ctx.freshName("days") + val year = ctx.freshName("year") + + val eval = child.gen(ctx) + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + long $utcTime = ${eval.primitive} / 10000; + long $days = $utcTime / 1000 / 3600 / 24; + int $year = (int) ($days / 365.24); + int $dayInYear = (int) ($days - $year * 365.24); + ${f(dayInYear, utcTime)} + } + """ + } + + def eval(input: InternalRow, f: (Int, Long) => Int): Any = { + val valueLeft = child.eval(input) + if (valueLeft == null) { + null + } else { + val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 + val days = utcTime / 1000 / 3600 / 24 + val year = days / 365.24 + val dayInYear = days - year.toInt * 365.24 + f(dayInYear.toInt, utcTime) + } + } + + override def toString: String = s"Year($child)" +} + + +case class Year(child: Expression) extends DateFormatExpression { + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 360) { + ${ev.primitive} = (int) (1970 + ($utc / 1000 / 3600 / 24 / 365.24)); + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.YEAR); + } + """) + } + + override def eval(input: InternalRow): Any = { + eval(input, (dayInYear, utcTime) => + if (dayInYear > 1 && dayInYear < 360) { + 1970 + (utcTime / 1000 / 3600 / 24 / 365.24).toInt + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.YEAR) + } + ) + } + + override def toString: String = s"Year($child)" +} + +case class Quarter(child: Expression) extends DateFormatExpression { + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 90) { + ${ev.primitive} = 1; + } else if ($day > 92 && $day < 181) { + ${ev.primitive} = 2; + } else if ($day > 183 && $day < 273) { + ${ev.primitive} = 3; + } else if ($day > 275 && $day < 364) { + ${ev.primitive} = 4; + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.MONTH) / 3 + 1; + } + """) + } + + override def eval(input: InternalRow): Any = { + eval(input, (dayInYear, utcTime) => + dayInYear match { + case i: Int if i > 1 && i < 89 => 1 + case i: Int if i > 93 && i < 180 => 2 + case i: Int if i > 184 && i < 272 => 3 + case i: Int if i > 276 && i < 362 => 4 + case _ => + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.MONTH) / 3 + 1 + } + ) + } + + override def toString: String = s"Quarter($child)" +} + +case class Month(child: Expression) extends DateFormatExpression { + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, utc) => + s""" + if ($day > 1 && $day < 30) { + ${ev.primitive} = 1; + } else if($day > 33 && $day < 58) { + ${ev.primitive} = 2; + } else if($day > 62 && $day < 89) { + ${ev.primitive} = 3; + } else if($day > 93 && $day < 119) { + ${ev.primitive} = 4; + } else if($day > 123 && $day < 150) { + ${ev.primitive} = 5; + } else if($day > 154 && $day < 180) { + ${ev.primitive} = 6; + } else if($day > 184 && $day < 211) { + ${ev.primitive} = 7; + } else if($day > 215 && $day < 242) { + ${ev.primitive} = 8; + } else if($day > 246 && $day < 272) { + ${ev.primitive} = 9; + } else if($day > 276 && $day < 303) { + ${ev.primitive} = 10; + } else if($day > 307 && $day < 333) { + ${ev.primitive} = 11; + } else if($day > 337 && $day < 362) { + ${ev.primitive} = 12; + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($utc); + ${ev.primitive} = c.get($cal.MONTH) + 1; + } + """) + } + + override def eval(input: InternalRow): Any = { + eval(input, (dayInYear, utcTime) => + dayInYear match { + case i: Int if i > 1 && i < 30 => 1 + case i: Int if i > 33 && i < 58 => 2 + case i: Int if i > 62 && i < 89 => 3 + case i: Int if i > 93 && i < 119 => 4 + case i: Int if i > 123 && i < 150 => 5 + case i: Int if i > 154 && i < 180 => 6 + case i: Int if i > 184 && i < 211 => 7 + case i: Int if i > 215 && i < 242 => 8 + case i: Int if i > 246 && i < 272 => 9 + case i: Int if i > 276 && i < 303 => 10 + case i: Int if i > 307 && i < 333 => 11 + case i: Int if i > 337 && i < 362 => 12 + case _ => + val c = Calendar.getInstance() + c.setTimeInMillis(utcTime) + c.get(Calendar.MONTH) + 1 + } + ) + } + + override def toString: String = s"Month($child)" +} + +case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val eval = child.gen(ctx) + + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $cal $c = $cal.getInstance(); + $c.setTimeInMillis(${eval.primitive} / 10000); + ${ev.primitive} = $c.get($cal.DAY_OF_MONTH); + } + """ + } + + override def eval(input: InternalRow): Any = { + val value = child.eval(input) + if (value == null) { + null + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) + c.get(Calendar.DAY_OF_MONTH) + } + } + + override def toString: String = s"Day($child)" +} + +case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = IntegerType + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val eval = child.gen(ctx) + + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + + ev.isNull = eval.isNull + eval.code + s""" + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + $cal $c = $cal.getInstance(); + $c.setTimeInMillis(${eval.primitive} / 10000); + ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); + } + """ + } + + override def eval(input: InternalRow): Any = { + val value = child.eval(input) + if (value == null) { + null + } else { + val c = Calendar.getInstance() + c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) + c.get(Calendar.WEEK_OF_YEAR) + } + } + + override def toString: String = s"WeekOfYear($child)" +} + + +case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression + with ExpectsInputTypes { + + override def dataType: DataType = StringType + + override def toString: String = s"DateFormat($left, $right)" + + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) + + override def eval(input: InternalRow): Any = { + val valueLeft = left.eval(input) + if (valueLeft == null) { + null + } else { + val valueRight = right.eval(input) + if (valueRight == null) { + null + } else { + val sdf = new SimpleDateFormat(valueRight.toString) + UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) + } + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val sdf = classOf[SimpleDateFormat].getName + defineCodeGen(ctx, ev, (x, y) => { + s"""${ctx.stringType}.fromString((new $sdf($y.toString())) + .format(new java.sql.Date($x / 10000)))""" + }) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index 1618c24871c60..ad2f9d30168da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -17,8 +17,13 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.util.Calendar + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types.{StringType, DateType, TimestampType} class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("datetime function current_date") { @@ -34,4 +39,137 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(math.abs(t1 - ct.getTime) < 5000) } + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + + test("DateFormat") { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } + + test("Year") { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) + checkEvaluation(Year(Literal(ts)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.YEAR)) + } + } + } + } + + test("Quarter") { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) + checkEvaluation(Quarter(Literal(ts)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) / 3 + 1) + } + } + } + } + + test("Month") { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) + checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) + checkEvaluation(Month(Literal(ts)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) + 1) + } + } + } + } + + test("Day") { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) + checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) + checkEvaluation(Day(Literal(ts)), 8) + } + + test("Hour") { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) + + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) + } + } + } + } + + test("Minute") { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) + } + } + } + + test("Seconds") { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) + } + } + + test("WeekOfYear") { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 45) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala index a4719a38de1d4..3875e09cf90c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat + +import org.apache.spark.sql.functions._ class DataFrameDateTimeSuite extends QueryTest { @@ -53,4 +56,151 @@ class DataFrameDateTimeSuite extends QueryTest { df.select("t").filter($"t" >= "2015"), Row(Date.valueOf("2015-01-01")) :: Nil) } + + + + test("date format") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), + Row("2015", "2015", "2013")) + + checkAnswer( + df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), + Row("2015", "2015", "2013")) + } + + test("year") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(year("a"), year("b"), year("c")), + Row(2015, 2015, 2013)) + + checkAnswer( + df.selectExpr("year(a)", "year(b)", "year(c)"), + Row(2015, 2015, 2013)) + } + + test("quarter") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(quarter("a"), quarter("b"), quarter("c")), + Row(2, 2, 4)) + + checkAnswer( + df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), + Row(2, 2, 4)) + } + + test("month") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(month("a"), month("b"), month("c")), + Row(4, 4, 4)) + + checkAnswer( + df.selectExpr("month(a)", "month(b)", "month(c)"), + Row(4, 4, 4)) + } + + test("day") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day("a"), day("b"), day("c")), + Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day(a)", "day(b)", "day(c)"), + Row(8, 8, 8)) + } + + test("hour") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(hour("a"), hour("b"), hour("c")), + Row(0, 13, 13)) + + checkAnswer( + df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + Row(0, 13, 13)) + } + + test("minute") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(minute("a"), minute("b"), minute("c")), + Row(0, 10, 10)) + + checkAnswer( + df.selectExpr("minute(a)", "minute(b)", "minute(c)"), + Row(0, 10, 10)) + } + + test("second") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(second("a"), second("b"), second("c")), + Row(0, 15, 15)) + + checkAnswer( + df.selectExpr("second(a)", "second(b)", "second(c)"), + Row(0, 15, 15)) + } + + test("weekOfYear") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), + Row(15, 15, 15)) + + checkAnswer( + df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), + Row(15, 15, 15)) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 77acf8d81eefa..84fc5e8630605 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -229,152 +229,6 @@ class DataFrameFunctionsSuite extends QueryTest { }) } - - - test("date format") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2013")) - - checkAnswer( - df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), - Row("2015", "2015", "2013")) - } - - test("year") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(year("a"), year("b"), year("c")), - Row(2015, 2015, 2013)) - - checkAnswer( - df.selectExpr("year(a)", "year(b)", "year(c)"), - Row(2015, 2015, 2013)) - } - - test("quarter") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(quarter("a"), quarter("b"), quarter("c")), - Row(2, 2, 4)) - - checkAnswer( - df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), - Row(2, 2, 4)) - } - - test("month") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(month("a"), month("b"), month("c")), - Row(4, 4, 4)) - - checkAnswer( - df.selectExpr("month(a)", "month(b)", "month(c)"), - Row(4, 4, 4)) - } - - test("day") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(day("a"), day("b"), day("c")), - Row(8, 8, 8)) - - checkAnswer( - df.selectExpr("day(a)", "day(b)", "day(c)"), - Row(8, 8, 8)) - } - - test("hour") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(hour("a"), hour("b"), hour("c")), - Row(0, 13, 13)) - - checkAnswer( - df.selectExpr("hour(a)", "hour(b)", "hour(c)"), - Row(0, 13, 13)) - } - - test("minute") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(minute("a"), minute("b"), minute("c")), - Row(0, 10, 10)) - - checkAnswer( - df.selectExpr("minute(a)", "minute(b)", "minute(c)"), - Row(0, 10, 10)) - } - - test("second") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(second("a"), second("b"), second("c")), - Row(0, 15, 15)) - - checkAnswer( - df.selectExpr("second(a)", "second(b)", "second(c)"), - Row(0, 15, 15)) - } - - test("weekOfYear") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), - Row(15, 15, 15)) - - checkAnswer( - df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), - Row(15, 15, 15)) - } - test("string ascii function") { val df = Seq(("abc", "")).toDF("a", "b") checkAnswer( From d9f8ac3e805b47421aa7d4f3d16d7a5267f5a64a Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:01:02 -0700 Subject: [PATCH 16/49] [SPARK-8199] implement fast track --- .../expressions/datetimeFunctions.scala | 344 ++++++++---------- .../expressions/DatetimeFunctionsSuite.scala | 217 +++++------ 2 files changed, 260 insertions(+), 301 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index ff5e0c6324ae0..07729385286a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -61,70 +61,55 @@ case class CurrentTimestamp() extends LeafExpression { } } +/** + * Abstract class for create time format expressions. + */ abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => - protected val format: Int + protected val factorToMilli: Int protected val cntPerInterval: Int override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - override def eval(input: InternalRow): Any = { - val valueLeft = child.eval(input) - if (valueLeft == null) { - null - } else { - val time = valueLeft.asInstanceOf[Long] / 10000 - val utcTime: Long = time + TimeZone.getDefault.getOffset(time) - ((utcTime / format) % cntPerInterval).toInt - } - } + override def dataType: DataType = IntegerType - override def genCode( - ctx: CodeGenContext, - ev: GeneratedExpressionCode): String = { + override protected def nullSafeEval(timestamp: Any): Any = { + val time = timestamp.asInstanceOf[Long] / 1000 + val longTime: Long = time + TimeZone.getDefault.getOffset(time) + ((longTime / factorToMilli) % cntPerInterval).toInt + } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val tz = classOf[TimeZone].getName - defineCodeGen(ctx, ev, (c) => s"""(${ctx.javaType(dataType)}) - ((($c / 10000) + $tz.getDefault().getOffset($c / 10000)) / $format % $cntPerInterval)""" + ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) + / $factorToMilli % $cntPerInterval)""" ) } } case class Hour(child: Expression) extends TimeFormatExpression { - override protected val format: Int = 1000 * 3600 + override protected val factorToMilli: Int = 1000 * 3600 override protected val cntPerInterval: Int = 24 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Hour($child)" } case class Minute(child: Expression) extends TimeFormatExpression { - override protected val format: Int = 1000 * 60 + override protected val factorToMilli: Int = 1000 * 60 override protected val cntPerInterval: Int = 60 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Minute($child)" } case class Second(child: Expression) extends TimeFormatExpression { - override protected val format: Int = 1000 + override protected val factorToMilli: Int = 1000 override protected val cntPerInterval: Int = 60 - - override def dataType: DataType = IntegerType - - override def toString: String = s"Second($child)" } abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { @@ -134,185 +119,181 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - protected def defineCodeGen( - ctx: CodeGenContext, - ev: GeneratedExpressionCode, - f: (String, String) => String): String = { - - val tz = classOf[TimeZone].getName - - val utcTime = ctx.freshName("utcTime") - val dayInYear = ctx.freshName("dayInYear") - val days = ctx.freshName("days") - val year = ctx.freshName("year") - - val eval = child.gen(ctx) - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - long $utcTime = ${eval.primitive} / 10000; - long $days = $utcTime / 1000 / 3600 / 24; - int $year = (int) ($days / 365.24); - int $dayInYear = (int) ($days - $year * 365.24); - ${f(dayInYear, utcTime)} - } - """ + protected def isLeapYear(year: Int): Boolean = { + year % 4 == 0 && (year % 100 != 0 || year % 400 == 0) } - def eval(input: InternalRow, f: (Int, Long) => Int): Any = { + def eval(input: InternalRow, f: (Int, Int, Long) => Int): Any = { val valueLeft = child.eval(input) if (valueLeft == null) { null } else { - val utcTime: Long = valueLeft.asInstanceOf[Long] / 10000 - val days = utcTime / 1000 / 3600 / 24 + val longTime: Long = valueLeft.asInstanceOf[Long] / 1000 + val days = longTime / 1000.0 / 3600.0 / 24.0 val year = days / 365.24 val dayInYear = days - year.toInt * 365.24 - f(dayInYear.toInt, utcTime) + f(dayInYear.toInt, 1970 + year.toInt, longTime) } } - override def toString: String = s"Year($child)" + protected def defineCodeGen( + ctx: CodeGenContext, + ev: GeneratedExpressionCode, + f: (String, String, String) => String): String = { + nullSafeCodeGen(ctx, ev, (date) => { + val longTime = ctx.freshName("longTime") + val dayInYear = ctx.freshName("dayInYear") + val days = ctx.freshName("days") + val year = ctx.freshName("year") + + s""" + long $longTime = $date / 1000; + long $days = $longTime / 1000 / 3600 / 24; + int $year = (int) ($days / 365.24); + int $dayInYear = (int) ($days - $year * 365.24); + $year += 1970; + ${f(dayInYear, year, longTime)} + """ + }) + } } case class Year(child: Expression) extends DateFormatExpression { - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => - s""" - if ($day > 1 && $day < 360) { - ${ev.primitive} = (int) (1970 + ($utc / 1000 / 3600 / 24 / 365.24)); - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); - ${ev.primitive} = c.get($cal.YEAR); - } - """) - } - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => + eval(input, (dayInYear, year, longTime) => if (dayInYear > 1 && dayInYear < 360) { - 1970 + (utcTime / 1000 / 3600 / 24 / 365.24).toInt + year } else { val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) + c.setTimeInMillis(longTime) c.get(Calendar.YEAR) } ) } - override def toString: String = s"Year($child)" + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val cal = classOf[Calendar].getName + defineCodeGen(ctx, ev, (day, year, longTime) => + s""" + if ($day > 1 && $day < 360) { + ${ev.primitive} = $year; + } else { + $cal c = $cal.getInstance(); + c.setTimeInMillis($longTime); + ${ev.primitive} = c.get($cal.YEAR); + } + """) + } } case class Quarter(child: Expression) extends DateFormatExpression { + override def eval(input: InternalRow): Any = { + eval(input, (dayInYear, year, longTime) => { + val leap = if (isLeapYear(year)) 1 else 0 + dayInYear match { + case i: Int if i > 3 && i < 88 + leap => 1 + case i: Int if i > 93 + leap && i < 179 + leap => 2 + case i: Int if i > 184 + leap && i < 271 + leap => 3 + case i: Int if i > 276 + leap && i < 363 + leap => 4 + case _ => + val c = Calendar.getInstance() + c.setTimeInMillis(longTime) + c.get(Calendar.MONTH) / 3 + 1 + } + }) + } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => + defineCodeGen(ctx, ev, (day, year, longTime) => { + val leap = ctx.freshName("leap") s""" - if ($day > 1 && $day < 90) { + int $leap = ($year % 4 == 0 && ($year % 100 != 0 || $year % 400 == 0)) ? 1 : 0; + if ($day > 3 && $day < 88 + $leap) { ${ev.primitive} = 1; - } else if ($day > 92 && $day < 181) { + } else if ($day > 93 + $leap && $day < 179 + $leap) { ${ev.primitive} = 2; - } else if ($day > 183 && $day < 273) { + } else if ($day > 184 + $leap && $day < 271 + $leap) { ${ev.primitive} = 3; - } else if ($day > 275 && $day < 364) { + } else if ($day > 276 + $leap && $day < 363 + $leap) { ${ev.primitive} = 4; } else { $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); + c.setTimeInMillis($longTime); ${ev.primitive} = c.get($cal.MONTH) / 3 + 1; } - """) + """}) } +} + +case class Month(child: Expression) extends DateFormatExpression { override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => + eval(input, (dayInYear, year, longTime) => { + val leap = if (isLeapYear(year)) 1 else 0 dayInYear match { - case i: Int if i > 1 && i < 89 => 1 - case i: Int if i > 93 && i < 180 => 2 - case i: Int if i > 184 && i < 272 => 3 - case i: Int if i > 276 && i < 362 => 4 + case i: Int if i > 3 && i < 29 => 1 + case i: Int if i > 34 && i < 57 + leap => 2 + case i: Int if i > 62 + leap && i < 88 + leap => 3 + case i: Int if i > 93 + leap && i < 118 + leap => 4 + case i: Int if i > 123 + leap && i < 149 + leap => 5 + case i: Int if i > 154 + leap && i < 179 + leap => 6 + case i: Int if i > 184 + leap && i < 210 + leap => 7 + case i: Int if i > 215 + leap && i < 241 + leap => 8 + case i: Int if i > 246 + leap && i < 271 + leap => 9 + case i: Int if i > 276 + leap && i < 302 + leap => 10 + case i: Int if i > 307 + leap && i < 332 + leap => 11 + case i: Int if i > 337 + leap && i < 363 + leap => 12 case _ => val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(Calendar.MONTH) / 3 + 1 + c.setTimeInMillis(longTime) + c.get(Calendar.MONTH) + 1 } - ) + }) } - override def toString: String = s"Quarter($child)" -} - -case class Month(child: Expression) extends DateFormatExpression { - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, utc) => + defineCodeGen(ctx, ev, (day, year, longTime) => { + val leap = ctx.freshName("leap") s""" - if ($day > 1 && $day < 30) { + int $leap = ($year % 4 == 0 && ($year % 100 != 0 || $year % 400 == 0)) ? 1 : 0; + if ($day > 3 && $day < 29) { ${ev.primitive} = 1; - } else if($day > 33 && $day < 58) { + } else if($day > 34 && $day < 57 + $leap) { ${ev.primitive} = 2; - } else if($day > 62 && $day < 89) { + } else if($day > 62 + $leap + $leap && $day < 88 + $leap) { ${ev.primitive} = 3; - } else if($day > 93 && $day < 119) { + } else if($day > 93 + $leap && $day < 118 + $leap) { ${ev.primitive} = 4; - } else if($day > 123 && $day < 150) { + } else if($day > 123 + $leap && $day < 149 + $leap) { ${ev.primitive} = 5; - } else if($day > 154 && $day < 180) { + } else if($day > 154 + $leap && $day < 179 + $leap) { ${ev.primitive} = 6; - } else if($day > 184 && $day < 211) { + } else if($day > 184 + $leap && $day < 210 + $leap) { ${ev.primitive} = 7; - } else if($day > 215 && $day < 242) { + } else if($day > 215 + $leap && $day < 241 + $leap) { ${ev.primitive} = 8; - } else if($day > 246 && $day < 272) { + } else if($day > 246 + $leap && $day < 271 + $leap) { ${ev.primitive} = 9; - } else if($day > 276 && $day < 303) { + } else if($day > 276 + $leap && $day < 302 + $leap) { ${ev.primitive} = 10; - } else if($day > 307 && $day < 333) { + } else if($day > 307 + $leap && $day < 332 + $leap) { ${ev.primitive} = 11; - } else if($day > 337 && $day < 362) { + } else if($day > 337 + $leap && $day < 363 + $leap) { ${ev.primitive} = 12; } else { $cal c = $cal.getInstance(); - c.setTimeInMillis($utc); + c.setTimeInMillis($longTime); ${ev.primitive} = c.get($cal.MONTH) + 1; } - """) - } - - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, utcTime) => - dayInYear match { - case i: Int if i > 1 && i < 30 => 1 - case i: Int if i > 33 && i < 58 => 2 - case i: Int if i > 62 && i < 89 => 3 - case i: Int if i > 93 && i < 119 => 4 - case i: Int if i > 123 && i < 150 => 5 - case i: Int if i > 154 && i < 180 => 6 - case i: Int if i > 184 && i < 211 => 7 - case i: Int if i > 215 && i < 242 => 8 - case i: Int if i > 246 && i < 272 => 9 - case i: Int if i > 276 && i < 303 => 10 - case i: Int if i > 307 && i < 333 => 11 - case i: Int if i > 337 && i < 362 => 12 - case _ => - val c = Calendar.getInstance() - c.setTimeInMillis(utcTime) - c.get(Calendar.MONTH) + 1 - } - ) + """}) } - - override def toString: String = s"Month($child)" } case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -322,20 +303,15 @@ case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val eval = child.gen(ctx) - - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { + nullSafeCodeGen(ctx, ev, (date) => { + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + s""" $cal $c = $cal.getInstance(); - $c.setTimeInMillis(${eval.primitive} / 10000); + $c.setTimeInMillis($date / 1000); ${ev.primitive} = $c.get($cal.DAY_OF_MONTH); - } - """ + """ + }) } override def eval(input: InternalRow): Any = { @@ -344,12 +320,10 @@ case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes null } else { val c = Calendar.getInstance() - c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) + c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 1000) c.get(Calendar.DAY_OF_MONTH) } } - - override def toString: String = s"Day($child)" } case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -358,67 +332,41 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInp override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val eval = child.gen(ctx) - - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - - ev.isNull = eval.isNull - eval.code + s""" - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = + nullSafeCodeGen(ctx, ev, (time) => { + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + s""" $cal $c = $cal.getInstance(); - $c.setTimeInMillis(${eval.primitive} / 10000); + $c.setTimeInMillis($time / 1000); ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); - } - """ - } + """ + }) - override def eval(input: InternalRow): Any = { - val value = child.eval(input) - if (value == null) { - null - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 10000) - c.get(Calendar.WEEK_OF_YEAR) - } + override protected def nullSafeEval(input: Any): Any = { + val c = Calendar.getInstance() + c.setTimeInMillis(input.asInstanceOf[Long] / 1000) + c.get(Calendar.WEEK_OF_YEAR) } - - override def toString: String = s"WeekOfYear($child)" } - case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression with ExpectsInputTypes { override def dataType: DataType = StringType - override def toString: String = s"DateFormat($left, $right)" - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) - override def eval(input: InternalRow): Any = { - val valueLeft = left.eval(input) - if (valueLeft == null) { - null - } else { - val valueRight = right.eval(input) - if (valueRight == null) { - null - } else { - val sdf = new SimpleDateFormat(valueRight.toString) - UTF8String.fromString(sdf.format(new Date(valueLeft.asInstanceOf[Long] / 10000))) - } - } + override protected def nullSafeEval(date: Any, format: Any): Any = { + val sdf = new SimpleDateFormat(format.toString) + UTF8String.fromString(sdf.format(new Date(date.asInstanceOf[Long] / 1000))) } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName - defineCodeGen(ctx, ev, (x, y) => { - s"""${ctx.stringType}.fromString((new $sdf($y.toString())) - .format(new java.sql.Date($x / 10000)))""" + defineCodeGen(ctx, ev, (date, format) => { + s"""${ctx.stringType}.fromString((new $sdf($format.toString())) + .format(new java.sql.Date($date / 1000)))""" }) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index ad2f9d30168da..a473f31fc545c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -19,13 +19,16 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.Calendar +import java.util.{TimeZone, Calendar} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types.{StringType, DateType, TimestampType} class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + val oldDefault = TimeZone.getDefault + test("datetime function current_date") { val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] @@ -43,133 +46,141 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - test("DateFormat") { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } + testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) + testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) + testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) + + def testWithTimezone(tz: TimeZone) { + TimeZone.setDefault(tz) + test("DateFormat - " + tz.getID) { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } - test("Year") { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) - checkEvaluation(Year(Literal(ts)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.YEAR)) + test("Year - " + tz.getID) { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) + checkEvaluation(Year(Literal(ts)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.YEAR)) + } } } } - } - test("Quarter") { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) - checkEvaluation(Quarter(Literal(ts)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) / 3 + 1) + test("Quarter - " + tz.getID) { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) + checkEvaluation(Quarter(Literal(ts)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) / 3 + 1) + } } } } - } - test("Month") { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) - checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) - checkEvaluation(Month(Literal(ts)), 11) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) + 1) + test("Month - " + tz.getID) { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) + checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) + checkEvaluation(Month(Literal(ts)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) + 1) + } } } } - } - test("Day") { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) - checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) - checkEvaluation(Day(Literal(ts)), 8) - } + test("Day - " + tz.getID) { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) + checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) + checkEvaluation(Day(Literal(ts)), 8) + } + + test("Hour - " + tz.getID) { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) + + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) + } + } + } + } - test("Hour") { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) + test("Minute - " + tz.getID) { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) } } } - } - test("Minute") { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) + test("Seconds - " + tz.getID) { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) } } - } - test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) + test("WeekOfYear - " + tz.getID) { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 45) } - } - test("WeekOfYear") { - checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 45) + TimeZone.setDefault(oldDefault) } } From 6f5d95c84697da7be3f005d82258e3a49f51d8fd Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:05:19 -0700 Subject: [PATCH 17/49] [SPARK-8199] fixed year interval --- .../spark/sql/catalyst/expressions/datetimeFunctions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 07729385286a9..dc874cc304a3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -163,7 +163,7 @@ case class Year(child: Expression) extends DateFormatExpression { override def eval(input: InternalRow): Any = { eval(input, (dayInYear, year, longTime) => - if (dayInYear > 1 && dayInYear < 360) { + if (dayInYear > 3 && dayInYear < 363) { year } else { val c = Calendar.getInstance() From f3e7a9fe08f80b2fd3ad1e3f7423abb5b20ffcd0 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:07:39 -0700 Subject: [PATCH 18/49] [SPARK-8199] revert change in DataFrameFunctionsSuite --- .../scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 7efcceee16c05..afba28515e032 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql -import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat - import org.apache.spark.sql.TestData._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ From 7d9f0eba0174418b93628dce5c82015e02b86a18 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:16:33 -0700 Subject: [PATCH 19/49] [SPARK-8199] git renaming issue --- .../expressions/DatetimeFunctionsSuite.scala | 234 ++++++++---------- 1 file changed, 105 insertions(+), 129 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index a473f31fc545c..c819ed62e6562 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -17,170 +17,146 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.{Date, Timestamp} +import java.sql.{Timestamp, Date} import java.text.SimpleDateFormat import java.util.{TimeZone, Calendar} import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.types.{StringType, DateType, TimestampType} +import org.apache.spark.sql.types.{TimestampType, StringType, DateType} -class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - - val oldDefault = TimeZone.getDefault - - test("datetime function current_date") { - val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) - val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] - val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis()) - assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) - } - - test("datetime function current_timestamp") { - val ct = DateTimeUtils.toJavaTimestamp(CurrentTimestamp().eval(EmptyRow).asInstanceOf[Long]) - val t1 = System.currentTimeMillis() - assert(math.abs(t1 - ct.getTime) < 5000) - } +class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) - testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) - testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) - - def testWithTimezone(tz: TimeZone) { - TimeZone.setDefault(tz) - test("DateFormat - " + tz.getID) { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } + test("DateFormat") { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } - test("Year - " + tz.getID) { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) - checkEvaluation(Year(Literal(ts)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.YEAR)) - } + test("Year") { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) + checkEvaluation(Year(Literal(ts)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.YEAR)) } } } + } - test("Quarter - " + tz.getID) { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) - checkEvaluation(Quarter(Literal(ts)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) / 3 + 1) - } + test("Quarter") { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) + checkEvaluation(Quarter(Literal(ts)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) / 3 + 1) } } } + } - test("Month - " + tz.getID) { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) - checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) - checkEvaluation(Month(Literal(ts)), 11) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) + 1) - } + test("Month") { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) + checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) + checkEvaluation(Month(Literal(ts)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) + 1) } } } + } - test("Day - " + tz.getID) { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) - checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) - checkEvaluation(Day(Literal(ts)), 8) - } - - test("Hour - " + tz.getID) { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) - - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) - } - } - } - } + test("Day") { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) + checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) + checkEvaluation(Day(Literal(ts)), 8) + } - test("Minute - " + tz.getID) { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) + test("Hour") { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) } } } + } - test("Seconds - " + tz.getID) { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) + test("Minute") { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) } } + } - test("WeekOfYear - " + tz.getID) { - checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 45) + test("Seconds") { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) } + } - TimeZone.setDefault(oldDefault) + test("WeekOfYear") { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 45) } } From ccb723c194df43719ddcd8dc0cfca8e920c6f41f Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:40:54 -0700 Subject: [PATCH 20/49] [SPARK-8199] style and fixed merge issues --- .../expressions/DatetimeFunctionsSuite.scala | 234 ++++++++++-------- .../apache/spark/sql/DataFrameDateSuite.scala | 150 ----------- .../spark/sql/DatetimeExpressionsSuite.scala | 149 +++++++++++ 3 files changed, 278 insertions(+), 255 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index c819ed62e6562..a473f31fc545c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -17,146 +17,170 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.{Timestamp, Date} +import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.{TimeZone, Calendar} import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.types.{TimestampType, StringType, DateType} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types.{StringType, DateType, TimestampType} -class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { +class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + val oldDefault = TimeZone.getDefault + + test("datetime function current_date") { + val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) + val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] + val d1 = DateTimeUtils.millisToDays(System.currentTimeMillis()) + assert(d0 <= cd && cd <= d1 && d1 - d0 <= 1) + } + + test("datetime function current_timestamp") { + val ct = DateTimeUtils.toJavaTimestamp(CurrentTimestamp().eval(EmptyRow).asInstanceOf[Long]) + val t1 = System.currentTimeMillis() + assert(math.abs(t1 - ct.getTime) < 5000) + } val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - test("DateFormat") { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } + testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) + testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) + testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) + + def testWithTimezone(tz: TimeZone) { + TimeZone.setDefault(tz) + test("DateFormat - " + tz.getID) { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } - test("Year") { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) - checkEvaluation(Year(Literal(ts)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.YEAR)) + test("Year - " + tz.getID) { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) + checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) + checkEvaluation(Year(Literal(ts)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.YEAR)) + } } } } - } - test("Quarter") { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) - checkEvaluation(Quarter(Literal(ts)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) / 3 + 1) + test("Quarter - " + tz.getID) { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) + checkEvaluation(Quarter(Literal(ts)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) / 3 + 1) + } } } } - } - test("Month") { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) - checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) - checkEvaluation(Month(Literal(ts)), 11) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) + 1) + test("Month - " + tz.getID) { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) + checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) + checkEvaluation(Month(Literal(ts)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MONTH) + 1) + } } } } - } - test("Day") { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) - checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) - checkEvaluation(Day(Literal(ts)), 8) - } + test("Day - " + tz.getID) { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) + checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) + checkEvaluation(Day(Literal(ts)), 8) + } - test("Hour") { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) + test("Hour - " + tz.getID) { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) + + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) + } + } + } + } - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => + test("Minute - " + tz.getID) { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) } } } - } - test("Minute") { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) + test("Seconds - " + tz.getID) { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) } } - } - test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) + test("WeekOfYear - " + tz.getID) { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(WeekOfYear(Literal(ts)), 45) } - } - test("WeekOfYear") { - checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 45) + TimeZone.setDefault(oldDefault) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala index 3875e09cf90c8..a4719a38de1d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameDateSuite.scala @@ -18,9 +18,6 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat - -import org.apache.spark.sql.functions._ class DataFrameDateTimeSuite extends QueryTest { @@ -56,151 +53,4 @@ class DataFrameDateTimeSuite extends QueryTest { df.select("t").filter($"t" >= "2015"), Row(Date.valueOf("2015-01-01")) :: Nil) } - - - - test("date format") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2013")) - - checkAnswer( - df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), - Row("2015", "2015", "2013")) - } - - test("year") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(year("a"), year("b"), year("c")), - Row(2015, 2015, 2013)) - - checkAnswer( - df.selectExpr("year(a)", "year(b)", "year(c)"), - Row(2015, 2015, 2013)) - } - - test("quarter") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(quarter("a"), quarter("b"), quarter("c")), - Row(2, 2, 4)) - - checkAnswer( - df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), - Row(2, 2, 4)) - } - - test("month") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(month("a"), month("b"), month("c")), - Row(4, 4, 4)) - - checkAnswer( - df.selectExpr("month(a)", "month(b)", "month(c)"), - Row(4, 4, 4)) - } - - test("day") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(day("a"), day("b"), day("c")), - Row(8, 8, 8)) - - checkAnswer( - df.selectExpr("day(a)", "day(b)", "day(c)"), - Row(8, 8, 8)) - } - - test("hour") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(hour("a"), hour("b"), hour("c")), - Row(0, 13, 13)) - - checkAnswer( - df.selectExpr("hour(a)", "hour(b)", "hour(c)"), - Row(0, 13, 13)) - } - - test("minute") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(minute("a"), minute("b"), minute("c")), - Row(0, 10, 10)) - - checkAnswer( - df.selectExpr("minute(a)", "minute(b)", "minute(c)"), - Row(0, 10, 10)) - } - - test("second") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(second("a"), second("b"), second("c")), - Row(0, 15, 15)) - - checkAnswer( - df.selectExpr("second(a)", "second(b)", "second(c)"), - Row(0, 15, 15)) - } - - test("weekOfYear") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), - Row(15, 15, 15)) - - checkAnswer( - df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), - Row(15, 15, 15)) - } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index 44b915304533c..441f0707655c8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.sql.{Timestamp, Date} +import java.text.SimpleDateFormat + import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ @@ -45,4 +48,150 @@ class DatetimeExpressionsSuite extends QueryTest { 0).getTime - System.currentTimeMillis()) < 5000) } + + + + test("date format") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), + Row("2015", "2015", "2013")) + + checkAnswer( + df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), + Row("2015", "2015", "2013")) + } + + test("year") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(year("a"), year("b"), year("c")), + Row(2015, 2015, 2013)) + + checkAnswer( + df.selectExpr("year(a)", "year(b)", "year(c)"), + Row(2015, 2015, 2013)) + } + + test("quarter") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(quarter("a"), quarter("b"), quarter("c")), + Row(2, 2, 4)) + + checkAnswer( + df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), + Row(2, 2, 4)) + } + + test("month") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(month("a"), month("b"), month("c")), + Row(4, 4, 4)) + + checkAnswer( + df.selectExpr("month(a)", "month(b)", "month(c)"), + Row(4, 4, 4)) + } + + test("day") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day("a"), day("b"), day("c")), + Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day(a)", "day(b)", "day(c)"), + Row(8, 8, 8)) + } + + test("hour") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(hour("a"), hour("b"), hour("c")), + Row(0, 13, 13)) + + checkAnswer( + df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + Row(0, 13, 13)) + } + + test("minute") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(minute("a"), minute("b"), minute("c")), + Row(0, 10, 10)) + + checkAnswer( + df.selectExpr("minute(a)", "minute(b)", "minute(c)"), + Row(0, 10, 10)) + } + + test("second") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(second("a"), second("b"), second("c")), + Row(0, 15, 15)) + + checkAnswer( + df.selectExpr("second(a)", "second(b)", "second(c)"), + Row(0, 15, 15)) + } + + test("weekOfYear") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), + Row(15, 15, 15)) + + checkAnswer( + df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), + Row(15, 15, 15)) + } } From c42b4447472e0430e1d1cecb3dfc490be89b8f87 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 9 Jul 2015 00:44:25 -0700 Subject: [PATCH 21/49] Removed merge conflict file Removed file that was created due to merge conflicts --- .../expressions/DateTimeFunctionsSuite.scala | 162 ------------------ 1 file changed, 162 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala deleted file mode 100644 index c819ed62e6562..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateTimeFunctionsSuite.scala +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import java.sql.{Timestamp, Date} -import java.text.SimpleDateFormat -import java.util.{TimeZone, Calendar} - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.types.{TimestampType, StringType, DateType} - -class DateTimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - - test("DateFormat") { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } - - test("Year") { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) - checkEvaluation(Year(Literal(ts)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.YEAR)) - } - } - } - } - - test("Quarter") { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) - checkEvaluation(Quarter(Literal(ts)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) / 3 + 1) - } - } - } - } - - test("Month") { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) - checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) - checkEvaluation(Month(Literal(ts)), 11) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) + 1) - } - } - } - } - - test("Day") { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) - checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) - checkEvaluation(Day(Literal(ts)), 8) - } - - test("Hour") { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) - - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) - } - } - } - } - - test("Minute") { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) - } - } - } - - test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) - } - } - - test("WeekOfYear") { - checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 45) - } - -} From ad17e9673a82ae79c76caf91cc8aa56679c5b103 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 10 Jul 2015 01:55:58 -0700 Subject: [PATCH 22/49] improved implementation --- .../expressions/datetimeFunctions.scala | 376 ++++++++++-------- .../expressions/DatetimeFunctionsSuite.scala | 163 ++++---- .../spark/sql/DatetimeExpressionsSuite.scala | 49 +-- 3 files changed, 294 insertions(+), 294 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index dc874cc304a3a..3bc8a5833b29c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -105,11 +105,41 @@ case class Minute(child: Expression) extends TimeFormatExpression { override protected val cntPerInterval: Int = 60 } -case class Second(child: Expression) extends TimeFormatExpression { +case class Second(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override protected val factorToMilli: Int = 1000 + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - override protected val cntPerInterval: Int = 60 + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(time: Any): Any = { + time.asInstanceOf[Long] / 1000L / 1000L % 60L + } + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, (time) => { + s"""${ev.primitive} = (${ctx.javaType(IntegerType)}) ($time / 1000L / 1000L % 60L);""" + }) + } +} + +private[sql] object DateFormatExpression { + + def isLeapYear(year: Int): Boolean = { + (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) + } + + val yearBoundaries = new Array[Int](2330 - 1599) + + var i = 0 + yearBoundaries(i) = -135140 + (1601 to 2330) foreach { year => + i = i + 1 + if (isLeapYear(year - 1)) { + yearBoundaries(i) = yearBoundaries(i - 1) + 366 + } else { + yearBoundaries(i) = yearBoundaries(i - 1) + 365 + } + } } abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { @@ -117,220 +147,218 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp override def dataType: DataType = IntegerType - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) - - protected def isLeapYear(year: Int): Boolean = { - year % 4 == 0 && (year % 100 != 0 || year % 400 == 0) - } + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - def eval(input: InternalRow, f: (Int, Int, Long) => Int): Any = { - val valueLeft = child.eval(input) - if (valueLeft == null) { - null - } else { - val longTime: Long = valueLeft.asInstanceOf[Long] / 1000 - val days = longTime / 1000.0 / 3600.0 / 24.0 - val year = days / 365.24 - val dayInYear = days - year.toInt * 365.24 - f(dayInYear.toInt, 1970 + year.toInt, longTime) + protected def calculateYearAndDayInYear(daysIn: Int): (Int, Int) = { + var index: Int = 370 + (daysIn / 365.24).toInt - 1 + while (DateFormatExpression.yearBoundaries(index) < daysIn + 1) { + index += 1 } + (index - 1 + 1600, daysIn - DateFormatExpression.yearBoundaries(index - 1) + 1) } - protected def defineCodeGen( - ctx: CodeGenContext, - ev: GeneratedExpressionCode, - f: (String, String, String) => String): String = { - nullSafeCodeGen(ctx, ev, (date) => { - val longTime = ctx.freshName("longTime") - val dayInYear = ctx.freshName("dayInYear") - val days = ctx.freshName("days") - val year = ctx.freshName("year") + protected def codeGen(ctx: CodeGenContext, ev: GeneratedExpressionCode, input: String, + f: (String, String) => String): String = { + val yb = ctx.freshName("yb") + val i = ctx.freshName("counter") + val x = ctx.freshName("counter") + val index = ctx.freshName("index") + val year = ctx.freshName("year") + val dayInYear = ctx.freshName("dayInYear") - s""" - long $longTime = $date / 1000; - long $days = $longTime / 1000 / 3600 / 24; - int $year = (int) ($days / 365.24); - int $dayInYear = (int) ($days - $year * 365.24); - $year += 1970; - ${f(dayInYear, year, longTime)} - """ - }) + s""" + int[] $yb = new int[2330 - 1599]; + $yb[0] = -135140; + int $x = 1; + for(int $i = 1601; $i <= 2330; $i++, $x++) { + $yb[$x] = (($i - 1) % 4 == 0 && (($i - 1) % 100 != 0 || ($i - 1) % 400 == 0)) ? + $yb[$x - 1] + 366 : $yb[$x - 1] + 365; + } + int $index = 370 + ((int) ($input / 365.24)) - 1; + while ($yb[$index] < $input + 1) { + $index++; + } + int $year = $index - 1 + 1600; + int $dayInYear = $input - $yb[$index - 1] + 1; + ${f(year, dayInYear)} + """ } } case class Year(child: Expression) extends DateFormatExpression { - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, year, longTime) => - if (dayInYear > 3 && dayInYear < 363) { - year - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(longTime) - c.get(Calendar.YEAR) - } - ) + override protected def nullSafeEval(input: Any): Any = { + calculateYearAndDayInYear(input.asInstanceOf[Int])._1 } - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, year, longTime) => - s""" - if ($day > 1 && $day < 360) { - ${ev.primitive} = $year; - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($longTime); - ${ev.primitive} = c.get($cal.YEAR); - } - """) + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, days => { + codeGen(ctx, ev, days, (year, dayInYear) => { + s"""${ev.primitive} = $year;""" + }) + }) } } case class Quarter(child: Expression) extends DateFormatExpression { - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, year, longTime) => { - val leap = if (isLeapYear(year)) 1 else 0 + override protected def nullSafeEval(input: Any): Any = { + val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) + val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 dayInYear match { - case i: Int if i > 3 && i < 88 + leap => 1 - case i: Int if i > 93 + leap && i < 179 + leap => 2 - case i: Int if i > 184 + leap && i < 271 + leap => 3 - case i: Int if i > 276 + leap && i < 363 + leap => 4 - case _ => - val c = Calendar.getInstance() - c.setTimeInMillis(longTime) - c.get(Calendar.MONTH) / 3 + 1 + case i: Int if i <= 90 + leap => 1 + case i: Int if i <= 181 + leap => 2 + case i: Int if i <= 273 + leap => 3 + case _ => 4 } - }) } - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, year, longTime) => { - val leap = ctx.freshName("leap") - s""" - int $leap = ($year % 4 == 0 && ($year % 100 != 0 || $year % 400 == 0)) ? 1 : 0; - if ($day > 3 && $day < 88 + $leap) { - ${ev.primitive} = 1; - } else if ($day > 93 + $leap && $day < 179 + $leap) { - ${ev.primitive} = 2; - } else if ($day > 184 + $leap && $day < 271 + $leap) { - ${ev.primitive} = 3; - } else if ($day > 276 + $leap && $day < 363 + $leap) { - ${ev.primitive} = 4; - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($longTime); - ${ev.primitive} = c.get($cal.MONTH) / 3 + 1; - } - """}) + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, days => { + codeGen(ctx, ev, days, (year, dayInYear) => { + s""" + if ($dayInYear <= 90) { + ${ev.primitive} = 1; + } else if ($dayInYear <= 181) { + ${ev.primitive} = 2; + } else if ($dayInYear <= 273) { + ${ev.primitive} = 3; + } else { + ${ev.primitive} = 4; + } + """ + }) + }) } } case class Month(child: Expression) extends DateFormatExpression { - override def eval(input: InternalRow): Any = { - eval(input, (dayInYear, year, longTime) => { - val leap = if (isLeapYear(year)) 1 else 0 - dayInYear match { - case i: Int if i > 3 && i < 29 => 1 - case i: Int if i > 34 && i < 57 + leap => 2 - case i: Int if i > 62 + leap && i < 88 + leap => 3 - case i: Int if i > 93 + leap && i < 118 + leap => 4 - case i: Int if i > 123 + leap && i < 149 + leap => 5 - case i: Int if i > 154 + leap && i < 179 + leap => 6 - case i: Int if i > 184 + leap && i < 210 + leap => 7 - case i: Int if i > 215 + leap && i < 241 + leap => 8 - case i: Int if i > 246 + leap && i < 271 + leap => 9 - case i: Int if i > 276 + leap && i < 302 + leap => 10 - case i: Int if i > 307 + leap && i < 332 + leap => 11 - case i: Int if i > 337 + leap && i < 363 + leap => 12 - case _ => - val c = Calendar.getInstance() - c.setTimeInMillis(longTime) - c.get(Calendar.MONTH) + 1 - } - }) + override protected def nullSafeEval(input: Any): Any = { + val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) + val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 + dayInYear match { + case i: Int if i <= 31 => 1 + case i: Int if i <= 59 + leap => 2 + case i: Int if i <= 90 + leap => 3 + case i: Int if i <= 120 + leap => 4 + case i: Int if i <= 151 + leap => 5 + case i: Int if i <= 181 + leap => 6 + case i: Int if i <= 212 + leap => 7 + case i: Int if i <= 243 + leap => 8 + case i: Int if i <= 273 + leap => 9 + case i: Int if i <= 304 + leap => 10 + case i: Int if i <= 334 + leap => 11 + case _ => 12 + } } - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - - val cal = classOf[Calendar].getName - defineCodeGen(ctx, ev, (day, year, longTime) => { - val leap = ctx.freshName("leap") - s""" - int $leap = ($year % 4 == 0 && ($year % 100 != 0 || $year % 400 == 0)) ? 1 : 0; - if ($day > 3 && $day < 29) { - ${ev.primitive} = 1; - } else if($day > 34 && $day < 57 + $leap) { - ${ev.primitive} = 2; - } else if($day > 62 + $leap + $leap && $day < 88 + $leap) { - ${ev.primitive} = 3; - } else if($day > 93 + $leap && $day < 118 + $leap) { - ${ev.primitive} = 4; - } else if($day > 123 + $leap && $day < 149 + $leap) { - ${ev.primitive} = 5; - } else if($day > 154 + $leap && $day < 179 + $leap) { - ${ev.primitive} = 6; - } else if($day > 184 + $leap && $day < 210 + $leap) { - ${ev.primitive} = 7; - } else if($day > 215 + $leap && $day < 241 + $leap) { - ${ev.primitive} = 8; - } else if($day > 246 + $leap && $day < 271 + $leap) { - ${ev.primitive} = 9; - } else if($day > 276 + $leap && $day < 302 + $leap) { - ${ev.primitive} = 10; - } else if($day > 307 + $leap && $day < 332 + $leap) { - ${ev.primitive} = 11; - } else if($day > 337 + $leap && $day < 363 + $leap) { - ${ev.primitive} = 12; - } else { - $cal c = $cal.getInstance(); - c.setTimeInMillis($longTime); - ${ev.primitive} = c.get($cal.MONTH) + 1; - } - """}) + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, days => { + codeGen(ctx, ev, days, (year, dayInYear) => { + val leap = ctx.freshName("leap") + s""" + int $leap = ($year % 4) == 0 && (($year % 100) != 0 || ($year % 400) == 0) ? 1 : 0; + if ($dayInYear <= 31) { + ${ev.primitive} = 1; + } else if ($dayInYear <= 59 + $leap) { + ${ev.primitive} = 2; + } else if ($dayInYear <= 90 + $leap) { + ${ev.primitive} = 3; + } else if ($dayInYear <= 120 + $leap) { + ${ev.primitive} = 4; + } else if ($dayInYear <= 151 + $leap) { + ${ev.primitive} = 5; + } else if ($dayInYear <= 181 + $leap) { + ${ev.primitive} = 6; + } else if ($dayInYear <= 212 + $leap) { + ${ev.primitive} = 7; + } else if ($dayInYear <= 243 + $leap) { + ${ev.primitive} = 8; + } else if ($dayInYear <= 273 + $leap) { + ${ev.primitive} = 9; + } else if ($dayInYear <= 304 + $leap) { + ${ev.primitive} = 10; + } else if ($dayInYear <= 334 + $leap) { + ${ev.primitive} = 11; + } else { + ${ev.primitive} = 12; + } + """ + }) + }) } } -case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, (date) => { - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - s""" - $cal $c = $cal.getInstance(); - $c.setTimeInMillis($date / 1000); - ${ev.primitive} = $c.get($cal.DAY_OF_MONTH); - """ - }) + override protected def nullSafeEval(input: Any): Any = { + val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) + val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 + dayInYear match { + case i: Int if i <= 31 => i + case i: Int if i <= 59 + leap => i - 31 + case i: Int if i <= 90 + leap => i - 59 + leap + case i: Int if i <= 120 + leap => i - 90 + leap + case i: Int if i <= 151 + leap => i - 120 + leap + case i: Int if i <= 181 + leap => i - 151 + leap + case i: Int if i <= 212 + leap => i - 181 + leap + case i: Int if i <= 243 + leap => i - 212 + leap + case i: Int if i <= 273 + leap => i - 243 + leap + case i: Int if i <= 304 + leap => i - 273 + leap + case i: Int if i <= 334 + leap => i - 304 + leap + case i: Int => i - 334 + leap + } } - override def eval(input: InternalRow): Any = { - val value = child.eval(input) - if (value == null) { - null - } else { - val c = Calendar.getInstance() - c.setTimeInMillis(child.eval(input).asInstanceOf[Long] / 1000) - c.get(Calendar.DAY_OF_MONTH) - } + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, days => { + codeGen(ctx, ev, days, (year, dayInYear) => { + val leap = ctx.freshName("leap") + s""" + int $leap = ($year % 4) == 0 && (($year % 100) != 0 || ($year % 400) == 0) ? 1 : 0; + if ($dayInYear <= 31) { + ${ev.primitive} = $dayInYear; + } else if ($dayInYear <= 59 + $leap) { + ${ev.primitive} = $dayInYear - 31; + } else if ($dayInYear <= 90 + $leap) { + ${ev.primitive} = $dayInYear - 59 + $leap; + } else if ($dayInYear <= 120 + $leap) { + ${ev.primitive} = $dayInYear - 90 + $leap; + } else if ($dayInYear <= 151 + $leap) { + ${ev.primitive} = $dayInYear - 120 + $leap; + } else if ($dayInYear <= 181 + $leap) { + ${ev.primitive} = $dayInYear - 151 + $leap; + } else if ($dayInYear <= 212 + $leap) { + ${ev.primitive} = $dayInYear - 181 + $leap; + } else if ($dayInYear <= 243 + $leap) { + ${ev.primitive} = $dayInYear - 212 + $leap; + } else if ($dayInYear <= 273 + $leap) { + ${ev.primitive} = $dayInYear - 243 + $leap; + } else if ($dayInYear <= 304 + $leap) { + ${ev.primitive} = $dayInYear - 273 + $leap; + } else if ($dayInYear <= 334 + $leap) { + ${ev.primitive} = $dayInYear - 304 + $leap; + } else { + ${ev.primitive} = $dayInYear - 334 + $leap; + } + """ + }) + }) } + } case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = IntegerType - override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = nullSafeCodeGen(ctx, ev, (time) => { @@ -338,14 +366,14 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInp val c = ctx.freshName("cal") s""" $cal $c = $cal.getInstance(); - $c.setTimeInMillis($time / 1000); + $c.setTimeInMillis($time * 1000L * 3600L * 24L); ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); """ }) override protected def nullSafeEval(input: Any): Any = { val c = Calendar.getInstance() - c.setTimeInMillis(input.asInstanceOf[Long] / 1000) + c.setTimeInMillis(input.asInstanceOf[Int] * 1000L * 3600L * 24L) c.get(Calendar.WEEK_OF_YEAR) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index a473f31fc545c..d585a5a4ccac8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -43,86 +43,101 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdfDate = new SimpleDateFormat("yyyy-MM-dd") val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) - testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) - testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) - - def testWithTimezone(tz: TimeZone) { - TimeZone.setDefault(tz) - test("DateFormat - " + tz.getID) { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + test("Year") { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), DateType)), 2015) + checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType)), 2015) + checkEvaluation(Year(Cast(Literal(ts), DateType)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (1 to 12 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.YEAR)) + } + } } + } - test("Year - " + tz.getID) { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), TimestampType)), 2015) - checkEvaluation(Year(Cast(Literal(sdf.format(d)), TimestampType)), 2015) - checkEvaluation(Year(Literal(ts)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.YEAR)) - } + test("Quarter") { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), DateType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType)), 2) + checkEvaluation(Quarter(Cast(Literal(ts), DateType)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12 by 3).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) / 3 + 1) } } } + } - test("Quarter - " + tz.getID) { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), TimestampType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdf.format(d)), TimestampType)), 2) - checkEvaluation(Quarter(Literal(ts)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) / 3 + 1) - } + test("Month") { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), DateType)), 4) + checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) + checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (1 to 12).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) + 1) } } } + } - test("Month - " + tz.getID) { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), TimestampType)), 4) - checkEvaluation(Month(Cast(Literal(sdf.format(d)), TimestampType)), 4) - checkEvaluation(Month(Literal(ts)), 11) + test("Day") { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), DateType)), 8) + checkEvaluation(Day(Cast(Literal(sdfDate.format(d)), DateType)), 8) + checkEvaluation(Day(Cast(Literal(ts), DateType)), 8) + } - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MONTH) + 1) - } - } - } + test("Seconds") { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) } + } - test("Day - " + tz.getID) { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), TimestampType)), 8) - checkEvaluation(Day(Cast(Literal(sdf.format(d)), TimestampType)), 8) - checkEvaluation(Day(Literal(ts)), 8) + testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) + testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) + testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) + + def testWithTimezone(tz: TimeZone) { + TimeZone.setDefault(tz) + test("DateFormat - " + tz.getID) { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") } test("Hour - " + tz.getID) { @@ -159,25 +174,11 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("Seconds - " + tz.getID) { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) - } - } - test("WeekOfYear - " + tz.getID) { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal(d), TimestampType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(WeekOfYear(Literal(ts)), 45) + checkEvaluation(WeekOfYear(Cast(Literal(d), DateType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) } TimeZone.setDefault(oldDefault) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index 441f0707655c8..ca90faeeb89b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -50,12 +50,13 @@ class DatetimeExpressionsSuite extends QueryTest { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - test("date format") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + test("date format") { val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( @@ -68,11 +69,7 @@ class DatetimeExpressionsSuite extends QueryTest { } test("year") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(year("a"), year("b"), year("c")), @@ -84,11 +81,9 @@ class DatetimeExpressionsSuite extends QueryTest { } test("quarter") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(quarter("a"), quarter("b"), quarter("c")), @@ -100,11 +95,7 @@ class DatetimeExpressionsSuite extends QueryTest { } test("month") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(month("a"), month("b"), month("c")), @@ -116,11 +107,7 @@ class DatetimeExpressionsSuite extends QueryTest { } test("day") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(day("a"), day("b"), day("c")), @@ -132,10 +119,6 @@ class DatetimeExpressionsSuite extends QueryTest { } test("hour") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( @@ -148,10 +131,6 @@ class DatetimeExpressionsSuite extends QueryTest { } test("minute") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( @@ -164,10 +143,6 @@ class DatetimeExpressionsSuite extends QueryTest { } test("second") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") checkAnswer( @@ -180,11 +155,7 @@ class DatetimeExpressionsSuite extends QueryTest { } test("weekOfYear") { - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") checkAnswer( df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), From f775f398a2867cdb54d2eb2885ca3a7dcfa234eb Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 10 Jul 2015 10:45:52 -0700 Subject: [PATCH 23/49] fixed return type --- .../spark/sql/catalyst/expressions/datetimeFunctions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 3bc8a5833b29c..b7d249a79f82e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -112,12 +112,12 @@ case class Second(child: Expression) extends UnaryExpression with ExpectsInputTy override def dataType: DataType = IntegerType override protected def nullSafeEval(time: Any): Any = { - time.asInstanceOf[Long] / 1000L / 1000L % 60L + (time.asInstanceOf[Long] / 1000L / 1000L % 60L).toInt } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, (time) => { - s"""${ev.primitive} = (${ctx.javaType(IntegerType)}) ($time / 1000L / 1000L % 60L);""" + s"""${ev.primitive} = (int) ($time / 1000L / 1000L % 60L);""" }) } } From 1a436c940012f01c6b3a792d458d8c656a86e2de Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Mon, 13 Jul 2015 00:00:41 -0700 Subject: [PATCH 24/49] wip --- .../expressions/datetimeFunctions.scala | 61 ++++++++----- .../expressions/DatetimeFunctionsSuite.scala | 85 +++++++++++++++++-- 2 files changed, 114 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index b7d249a79f82e..8582ffc81faae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -145,41 +145,56 @@ private[sql] object DateFormatExpression { abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => + val daysIn400Year: Int = (365.2425 * 400).toInt + + val to2001 = -11323 + val to1601 = to2001 + daysIn400Year + + private[this] def numYears(i: Int): Int = { + (i / 365.2425).toInt + } + override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(DateType) protected def calculateYearAndDayInYear(daysIn: Int): (Int, Int) = { - var index: Int = 370 + (daysIn / 365.24).toInt - 1 - while (DateFormatExpression.yearBoundaries(index) < daysIn + 1) { - index += 1 + val daysNormalized = daysIn + to1601 + val numOfQuarterCenturies = daysNormalized / daysIn400Year + val daysIn400 = daysNormalized % daysIn400Year + 1 + val years = numYears(daysIn400) + if (years == 400) { + (1601 + 400 * numOfQuarterCenturies + 399, 365) + } else { + val leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) + val year: Int = 1601 + 400 * numOfQuarterCenturies + (daysIn400 - leapDays) / 365 + var dayInYear = (daysIn400 - leapDays) % 365 + if (dayInYear == 0 && year % 400 == 0) dayInYear = 365 + (year, dayInYear) } - (index - 1 + 1600, daysIn - DateFormatExpression.yearBoundaries(index - 1) + 1) } protected def codeGen(ctx: CodeGenContext, ev: GeneratedExpressionCode, input: String, f: (String, String) => String): String = { - val yb = ctx.freshName("yb") - val i = ctx.freshName("counter") - val x = ctx.freshName("counter") - val index = ctx.freshName("index") val year = ctx.freshName("year") val dayInYear = ctx.freshName("dayInYear") s""" - int[] $yb = new int[2330 - 1599]; - $yb[0] = -135140; - int $x = 1; - for(int $i = 1601; $i <= 2330; $i++, $x++) { - $yb[$x] = (($i - 1) % 4 == 0 && (($i - 1) % 100 != 0 || ($i - 1) % 400 == 0)) ? - $yb[$x - 1] + 366 : $yb[$x - 1] + 365; - } - int $index = 370 + ((int) ($input / 365.24)) - 1; - while ($yb[$index] < $input + 1) { - $index++; - } - int $year = $index - 1 + 1600; - int $dayInYear = $input - $yb[$index - 1] + 1; + int daysIn400Year = (int) (365.2425 * 400); + + int to2001 = -11323; + int to1601 = to2001 + daysIn400Year; + + int daysNormalized = $input + to1601; + int numOfQuarterCenturies = daysNormalized / daysIn400Year; + int daysIn400 = daysNormalized % daysIn400Year; + int years = daysIn400 / 365; + int extra = ((years % 4) == 0 && years != 0) ? 1 : 0; + int leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) + extra; + int dayInYear = (daysIn400 - leapDays) % 365; + int year = 1601 + (400 * numOfQuarterCenturies) + (daysIn400 - leapDays) / 365; + int $year = year; + int $dayInYear = dayInYear + 1; ${f(year, dayInYear)} """ } @@ -254,7 +269,7 @@ case class Month(child: Expression) extends DateFormatExpression { } } - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + /*override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, days => { codeGen(ctx, ev, days, (year, dayInYear) => { val leap = ctx.freshName("leap") @@ -288,7 +303,7 @@ case class Month(child: Expression) extends DateFormatExpression { """ }) }) - } + }*/ } case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index d585a5a4ccac8..586e48c106b51 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -55,7 +55,7 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val c = Calendar.getInstance() (2000 to 2010).foreach { y => - (1 to 12 by 11).foreach { m => + (0 to 11 by 11).foreach { m => c.set(y, m, 28) (0 to 5 * 24).foreach { i => c.add(Calendar.HOUR_OF_DAY, 1) @@ -66,6 +66,21 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("millis to date") { + var l: Long = 970383600696L + var lBase = l + TimeZone.getDefault.getOffset(l) + System.out.println(new Date(l)) + assert(DateTimeUtils.millisToDays(l) == lBase / 3600 / 24 / 1000) + + l = 970383600977L + lBase = l + TimeZone.getDefault.getOffset(l) + System.out.println(new Date(l)) + System.out.println(DateTimeUtils.millisToDays(l)) + println(new SimpleDateFormat("D").format(new Date(l))) + println(lBase / 3600 / 24 / 1000) + assert(DateTimeUtils.millisToDays(l) == lBase / 3600 / 24 / 1000) + } + test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Cast(Literal(d), DateType)), 2) @@ -74,8 +89,8 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val c = Calendar.getInstance() (2003 to 2004).foreach { y => - (1 to 12 by 3).foreach { m => - c.set(y, m, 28) + (0 to 11 by 3).foreach { m => + c.set(y, m, 28, 0, 0, 0) (0 to 5 * 24).foreach { i => c.add(Calendar.HOUR_OF_DAY, 1) checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), @@ -86,17 +101,69 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Month") { - checkEvaluation(Month(Literal.create(null, DateType)), null) + /*checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Cast(Literal(d), DateType)), 4) checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) - checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) + checkEvaluation(Month(Cast(Literal(ts), DateType)), 11)*/ + + val x = Calendar.getInstance() + + x.setTimeInMillis(946713600707L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) + + x.setTimeInMillis(1072944000077L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) + + x.setTimeInMillis(981014400345L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) + + x.setTimeInMillis(978249600433L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) + + x.setTimeInMillis(970383600000L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) + + x.setTimeInMillis(954489600409L) + println(x.getTime) + println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) + checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), + x.get(Calendar.MONTH) + 1) - val c = Calendar.getInstance() (2003 to 2004).foreach { y => - (1 to 12).foreach { m => - c.set(y, m, 28) + (0 to 11).foreach { m => (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.HOUR_OF_DAY, i) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) + 1) + } + } + } + + val sdf = new SimpleDateFormat("D") + (1998 to 2005).foreach { y => + (0 to 11).foreach { m => + (0 to 5 * 24).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.HOUR_OF_DAY, i) + println(Cast(Literal(new Date(c.getTimeInMillis)), DateType).eval(null) + " " + sdf.format(c.getTime) + " " + c.getTimeInMillis) checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), c.get(Calendar.MONTH) + 1) } From 4fb66da3d80b29ac571cee877cbf95c77d2005cf Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Mon, 13 Jul 2015 07:47:28 -0700 Subject: [PATCH 25/49] WIP: date functions on calculation only --- .../catalyst/expressions/datetimeFunctions.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 8582ffc81faae..50464deb73d3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -163,15 +163,11 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp val numOfQuarterCenturies = daysNormalized / daysIn400Year val daysIn400 = daysNormalized % daysIn400Year + 1 val years = numYears(daysIn400) - if (years == 400) { - (1601 + 400 * numOfQuarterCenturies + 399, 365) - } else { - val leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) - val year: Int = 1601 + 400 * numOfQuarterCenturies + (daysIn400 - leapDays) / 365 - var dayInYear = (daysIn400 - leapDays) % 365 - if (dayInYear == 0 && year % 400 == 0) dayInYear = 365 - (year, dayInYear) - } + val leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) + val year: Int = 1601 + 400 * numOfQuarterCenturies + (daysIn400 - leapDays) / 365 + var dayInYear = (daysIn400 - leapDays) % 365 + if (dayInYear == 0 && year % 400 == 0) dayInYear = 365 + (year, dayInYear) } protected def codeGen(ctx: CodeGenContext, ev: GeneratedExpressionCode, input: String, From 740af0e5ed7983c3128a56b55585ab7a4f0e99d7 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Mon, 13 Jul 2015 15:41:20 -0700 Subject: [PATCH 26/49] implement date function using a calculation based on days --- python/pyspark/sql/functions.py | 13 ++ .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/datetimeFunctions.scala | 101 ++++++++------ .../expressions/DatetimeFunctionsSuite.scala | 123 ++++++++++-------- .../org/apache/spark/sql/functions.scala | 18 +++ .../spark/sql/DatetimeExpressionsSuite.scala | 12 ++ 6 files changed, 168 insertions(+), 100 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index b358e656d1b3e..2161a9904a161 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -642,6 +642,7 @@ def dateFormat(dateCol, formatCol): """ Convert the given date into the format specified by the second argument. Return type is always string. + NOTE: Prefer using functions like year. These use an optimized implementation. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(dateFormat('a', 'MM/dd/yyy').alias('date')).collect() @@ -701,6 +702,18 @@ def day(col): return Column(sc._jvm.functions.day(col)) +@since(1.5) +def dayInYear(col): + """ + Extract the day of the year of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(dayInYear('a').alias('day')).collect() + [Row(day=128)] + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.day(col)) + + @since(1.5) def hour(col): """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 09f7d5635ad6c..c534192ee4833 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -166,6 +166,7 @@ object FunctionRegistry { expression[Quarter]("quarter"), expression[Month]("month"), expression[Day]("day"), + expression[DayInYear]("dayinyear"), expression[Hour]("hour"), expression[Minute]("minute"), expression[Second]("second"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 50464deb73d3e..1b00d9ad12cd1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -128,30 +128,25 @@ private[sql] object DateFormatExpression { (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) } - val yearBoundaries = new Array[Int](2330 - 1599) - - var i = 0 - yearBoundaries(i) = -135140 - (1601 to 2330) foreach { year => - i = i + 1 - if (isLeapYear(year - 1)) { - yearBoundaries(i) = yearBoundaries(i - 1) + 366 - } else { - yearBoundaries(i) = yearBoundaries(i - 1) + 365 - } - } + } abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => - val daysIn400Year: Int = (365.2425 * 400).toInt - + val daysIn400Years: Int = 146097 val to2001 = -11323 - val to1601 = to2001 + daysIn400Year - private[this] def numYears(i: Int): Int = { - (i / 365.2425).toInt + // this is year -17999, calculation: 50 * daysIn400Year + val toYearZero = to2001 + 7304850 + + private[this] def yearBoundary(year: Int): Int = { + year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) + } + + private[this] def numYears(in: Int): Int = { + val year = in / 365 + if (in > yearBoundary(year)) year else year - 1 } override def dataType: DataType = IntegerType @@ -159,43 +154,63 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp override def inputTypes: Seq[AbstractDataType] = Seq(DateType) protected def calculateYearAndDayInYear(daysIn: Int): (Int, Int) = { - val daysNormalized = daysIn + to1601 - val numOfQuarterCenturies = daysNormalized / daysIn400Year - val daysIn400 = daysNormalized % daysIn400Year + 1 - val years = numYears(daysIn400) - val leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) - val year: Int = 1601 + 400 * numOfQuarterCenturies + (daysIn400 - leapDays) / 365 - var dayInYear = (daysIn400 - leapDays) % 365 - if (dayInYear == 0 && year % 400 == 0) dayInYear = 365 + val daysNormalized = daysIn + toYearZero + val numOfQuarterCenturies = daysNormalized / daysIn400Years + val daysInThis400 = daysNormalized % daysIn400Years + 1 + val years = numYears(daysInThis400) + val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years + val dayInYear = daysInThis400 - yearBoundary(years) (year, dayInYear) } protected def codeGen(ctx: CodeGenContext, ev: GeneratedExpressionCode, input: String, f: (String, String) => String): String = { + val daysIn400Years = ctx.freshName("daysIn400Years") + val to2001 = ctx.freshName("to2001") + val toYearZero = ctx.freshName("toYearZero") + val daysNormalized = ctx.freshName("daysNormalized") + val numOfQuarterCenturies = ctx.freshName("numOfQuarterCenturies") + val daysInThis400 = ctx.freshName("daysInThis400") + val years = ctx.freshName("years") val year = ctx.freshName("year") val dayInYear = ctx.freshName("dayInYear") s""" - int daysIn400Year = (int) (365.2425 * 400); - - int to2001 = -11323; - int to1601 = to2001 + daysIn400Year; - - int daysNormalized = $input + to1601; - int numOfQuarterCenturies = daysNormalized / daysIn400Year; - int daysIn400 = daysNormalized % daysIn400Year; - int years = daysIn400 / 365; - int extra = ((years % 4) == 0 && years != 0) ? 1 : 0; - int leapDays = ((years - 1) / 4) - (((years - 1) / 100) - ((years - 1) / 400)) + extra; - int dayInYear = (daysIn400 - leapDays) % 365; - int year = 1601 + (400 * numOfQuarterCenturies) + (daysIn400 - leapDays) / 365; - int $year = year; - int $dayInYear = dayInYear + 1; - ${f(year, dayInYear)} + int $daysIn400Years = 146097; + int $to2001 = -11323; + int $toYearZero = to2001 + 7304850; + + int $daysNormalized = $input + $toYearZero; + int $numOfQuarterCenturies = $daysNormalized / $daysIn400Years; + int $daysInThis400 = $daysNormalized % $daysIn400Years + 1; + int $years = $daysInThis400 / 365; + + $years = ($daysInThis400 > $years * 365 + (($years / 4 ) - ($years / 100) + + ($years / 400))) ? $years : $years - 1; + + int $year = (2001 - 20000) + 400 * $numOfQuarterCenturies + years; + int $dayInYear = $daysInThis400 - + ($years * 365 + (($years / 4 ) - ($years / 100) + ($years / 400))); + ${f(year, dayInYear)}; """ } } +case class DayInYear(child: Expression) extends DateFormatExpression { + + override protected def nullSafeEval(input: Any): Any = { + calculateYearAndDayInYear(input.asInstanceOf[Int])._2 + } + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, days => { + codeGen(ctx, ev, days, (year, dayInYear) => { + s"""${ev.primitive} = $dayInYear;""" + }) + }) + } +} + case class Year(child: Expression) extends DateFormatExpression { @@ -265,7 +280,7 @@ case class Month(child: Expression) extends DateFormatExpression { } } - /*override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, days => { codeGen(ctx, ev, days, (year, dayInYear) => { val leap = ctx.freshName("leap") @@ -299,7 +314,7 @@ case class Month(child: Expression) extends DateFormatExpression { """ }) }) - }*/ + } } case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index 586e48c106b51..abfd2d71d77e8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -47,6 +47,69 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + test("Day in Year") { + val sdfDay = new SimpleDateFormat("D") + (2002 to 2004).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (1998 to 2002).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (1969 to 1970).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (2402 to 2404).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (2398 to 2402).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + } + test("Year") { checkEvaluation(Year(Literal.create(null, DateType)), null) checkEvaluation(Year(Cast(Literal(d), DateType)), 2015) @@ -66,21 +129,6 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("millis to date") { - var l: Long = 970383600696L - var lBase = l + TimeZone.getDefault.getOffset(l) - System.out.println(new Date(l)) - assert(DateTimeUtils.millisToDays(l) == lBase / 3600 / 24 / 1000) - - l = 970383600977L - lBase = l + TimeZone.getDefault.getOffset(l) - System.out.println(new Date(l)) - System.out.println(DateTimeUtils.millisToDays(l)) - println(new SimpleDateFormat("D").format(new Date(l))) - println(lBase / 3600 / 24 / 1000) - assert(DateTimeUtils.millisToDays(l) == lBase / 3600 / 24 / 1000) - } - test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Cast(Literal(d), DateType)), 2) @@ -101,48 +149,10 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Month") { - /*checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Cast(Literal(d), DateType)), 4) checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) - checkEvaluation(Month(Cast(Literal(ts), DateType)), 11)*/ - - val x = Calendar.getInstance() - - x.setTimeInMillis(946713600707L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) - - x.setTimeInMillis(1072944000077L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) - - x.setTimeInMillis(981014400345L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) - - x.setTimeInMillis(978249600433L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) - - x.setTimeInMillis(970383600000L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) - - x.setTimeInMillis(954489600409L) - println(x.getTime) - println(Cast(Literal(new Date(x.getTimeInMillis)), DateType).eval(null)) - checkEvaluation(Month(Cast(Literal(new Date(x.getTimeInMillis)), DateType)), - x.get(Calendar.MONTH) + 1) + checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) (2003 to 2004).foreach { y => (0 to 11).foreach { m => @@ -157,13 +167,12 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } val sdf = new SimpleDateFormat("D") - (1998 to 2005).foreach { y => + (1999 to 2000).foreach { y => (0 to 11).foreach { m => (0 to 5 * 24).foreach { i => val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) c.add(Calendar.HOUR_OF_DAY, i) - println(Cast(Literal(new Date(c.getTimeInMillis)), DateType).eval(null) + " " + sdf.format(c.getTime) + " " + c.getTimeInMillis) checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), c.get(Calendar.MONTH) + 1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 152d7ec7ab835..a02f45dbb52d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1602,6 +1602,8 @@ object functions { /** * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument + * + * NOTE: Prefer using functions like year. These use an optimized implementation. * @group datetime_funcs * @since 1.5.0 */ @@ -1610,6 +1612,8 @@ object functions { /** * Converts a date/timestamp/string to a value of string in the format specified by the date * format given by the second argument + * + * NOTE: Prefer using functions like year. These use an optimized implementation. * @group datetime_funcs * @since 1.5.0 */ @@ -1671,6 +1675,20 @@ object functions { */ def day(columnName: String): Column = day(Column(columnName)) + /** + * Extracts the day of the year as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def dayInYear(e: Column): Column = DayInYear(e.expr) + + /** + * Extracts the day of the year as an integer from a given date/timestamp/string + * @group datetime_funcs + * @since 1.5.0 + */ + def dayInYear(columnName: String): Column = dayInYear(Column(columnName)) + /** * Extracts the hours as an integer from a given date/timestamp/string * @group datetime_funcs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index ca90faeeb89b3..4f03500f184c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -118,6 +118,18 @@ class DatetimeExpressionsSuite extends QueryTest { Row(8, 8, 8)) } + test("dayInYear") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(dayInYear("a"), dayInYear("b"), dayInYear("c")), + Row(128, 128, 128)) + + checkAnswer( + df.selectExpr("dayInYear(a)", "dayInYear(b)", "dayInYear(c)"), + Row(128, 128, 128)) + } + test("hour") { val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") From ec87c69ae7896db9de64dc2d865140fe7374f5ae Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 11:46:29 -0700 Subject: [PATCH 27/49] [SPARK-8119] bug fixing and refactoring --- .../expressions/datetimeFunctions.scala | 65 +++++++++---------- 1 file changed, 29 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 1b00d9ad12cd1..ce3a1d982d47d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -61,15 +61,7 @@ case class CurrentTimestamp() extends LeafExpression { } } -/** - * Abstract class for create time format expressions. - */ -abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTypes { - self: Product => - - protected val factorToMilli: Int - - protected val cntPerInterval: Int +case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -77,32 +69,38 @@ abstract class TimeFormatExpression extends UnaryExpression with ExpectsInputTyp override protected def nullSafeEval(timestamp: Any): Any = { val time = timestamp.asInstanceOf[Long] / 1000 - val longTime: Long = time + TimeZone.getDefault.getOffset(time) - ((longTime / factorToMilli) % cntPerInterval).toInt + val longTime: Long = time.asInstanceOf[Long] + TimeZone.getDefault.getOffset(time) + ((longTime / (1000 * 3600)) % 24).toInt } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val tz = classOf[TimeZone].getName defineCodeGen(ctx, ev, (c) => - s"""(${ctx.javaType(dataType)}) - ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) - / $factorToMilli % $cntPerInterval)""" + s"""(int) ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) + / (1000 * 3600) % 24)""".stripMargin ) } } -case class Hour(child: Expression) extends TimeFormatExpression { - - override protected val factorToMilli: Int = 1000 * 3600 +case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override protected val cntPerInterval: Int = 24 -} + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) -case class Minute(child: Expression) extends TimeFormatExpression { + override def dataType: DataType = IntegerType - override protected val factorToMilli: Int = 1000 * 60 + override protected def nullSafeEval(timestamp: Any): Any = { + val time = timestamp.asInstanceOf[Long] / 1000 + val longTime: Long = time.asInstanceOf[Long] + TimeZone.getDefault.getOffset(time) + ((longTime / (1000 * 60)) % 60).toInt + } - override protected val cntPerInterval: Int = 60 + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val tz = classOf[TimeZone].getName + defineCodeGen(ctx, ev, (c) => + s"""(int) ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) + / (1000 * 60) % 60)""".stripMargin + ) + } } case class Second(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -122,15 +120,6 @@ case class Second(child: Expression) extends UnaryExpression with ExpectsInputTy } } -private[sql] object DateFormatExpression { - - def isLeapYear(year: Int): Boolean = { - (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) - } - - -} - abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { self: Product => @@ -140,6 +129,10 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp // this is year -17999, calculation: 50 * daysIn400Year val toYearZero = to2001 + 7304850 + protected def isLeapYear(year: Int): Boolean = { + (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) + } + private[this] def yearBoundary(year: Int): Int = { year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) } @@ -178,7 +171,7 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp s""" int $daysIn400Years = 146097; int $to2001 = -11323; - int $toYearZero = to2001 + 7304850; + int $toYearZero = $to2001 + 7304850; int $daysNormalized = $input + $toYearZero; int $numOfQuarterCenturies = $daysNormalized / $daysIn400Years; @@ -188,7 +181,7 @@ abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTyp $years = ($daysInThis400 > $years * 365 + (($years / 4 ) - ($years / 100) + ($years / 400))) ? $years : $years - 1; - int $year = (2001 - 20000) + 400 * $numOfQuarterCenturies + years; + int $year = (2001 - 20000) + 400 * $numOfQuarterCenturies + $years; int $dayInYear = $daysInThis400 - ($years * 365 + (($years / 4 ) - ($years / 100) + ($years / 400))); ${f(year, dayInYear)}; @@ -231,7 +224,7 @@ case class Quarter(child: Expression) extends DateFormatExpression { override protected def nullSafeEval(input: Any): Any = { val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 + val leap = if (isLeapYear(year)) 1 else 0 dayInYear match { case i: Int if i <= 90 + leap => 1 case i: Int if i <= 181 + leap => 2 @@ -263,7 +256,7 @@ case class Month(child: Expression) extends DateFormatExpression { override protected def nullSafeEval(input: Any): Any = { val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 + val leap = if (isLeapYear(year)) 1 else 0 dayInYear match { case i: Int if i <= 31 => 1 case i: Int if i <= 59 + leap => 2 @@ -325,7 +318,7 @@ case class Day(child: Expression) extends DateFormatExpression with ExpectsInput override protected def nullSafeEval(input: Any): Any = { val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (DateFormatExpression.isLeapYear(year)) 1 else 0 + val leap = if (isLeapYear(year)) 1 else 0 dayInYear match { case i: Int if i <= 31 => i case i: Int if i <= 59 + leap => i - 31 From 08526554ce6e5fa3b20364813dfe520a212089cc Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 12:12:09 -0700 Subject: [PATCH 28/49] [SPARK-8119] changed from ExpectsInputTypes to implicit casts --- python/pyspark/sql/functions.py | 2 +- .../catalyst/expressions/datetimeFunctions.scala | 14 +++++++------- .../spark/sql/DatetimeExpressionsSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 912a87cd9ecfd..4a3744295da57 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -723,7 +723,7 @@ def dayInYear(col): Extract the day of the year of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(dayInYear('a').alias('day')).collect() - [Row(day=128)] + [Row(day=98)] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.day(col)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index ce3a1d982d47d..e162110934738 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -61,7 +61,7 @@ case class CurrentTimestamp() extends LeafExpression { } } -case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -82,7 +82,7 @@ case class Hour(child: Expression) extends UnaryExpression with ExpectsInputType } } -case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Minute(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -103,7 +103,7 @@ case class Minute(child: Expression) extends UnaryExpression with ExpectsInputTy } } -case class Second(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Second(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType) @@ -120,7 +120,7 @@ case class Second(child: Expression) extends UnaryExpression with ExpectsInputTy } } -abstract class DateFormatExpression extends UnaryExpression with ExpectsInputTypes { +abstract class DateFormatExpression extends UnaryExpression with ImplicitCastInputTypes { self: Product => val daysIn400Years: Int = 146097 @@ -310,7 +310,7 @@ case class Month(child: Expression) extends DateFormatExpression { } } -case class Day(child: Expression) extends DateFormatExpression with ExpectsInputTypes { +case class Day(child: Expression) extends DateFormatExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType @@ -373,7 +373,7 @@ case class Day(child: Expression) extends DateFormatExpression with ExpectsInput } -case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType @@ -398,7 +398,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ExpectsInp } case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression - with ExpectsInputTypes { + with ImplicitCastInputTypes { override def dataType: DataType = StringType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index 4f03500f184c7..902d4c73ad09f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -123,11 +123,11 @@ class DatetimeExpressionsSuite extends QueryTest { checkAnswer( df.select(dayInYear("a"), dayInYear("b"), dayInYear("c")), - Row(128, 128, 128)) + Row(98, 98, 98)) checkAnswer( df.selectExpr("dayInYear(a)", "dayInYear(b)", "dayInYear(c)"), - Row(128, 128, 128)) + Row(98, 98, 98)) } test("hour") { From 1b2e54056a9e0924f1a6f2a076e4ee4b50024775 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 14:05:56 -0700 Subject: [PATCH 29/49] [SPARK-8119] style fix --- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index e2563f7df2a77..dd50b99400992 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1699,7 +1699,7 @@ object functions { * @since 1.5.0 */ def length(columnName: String): Column = length(Column(columnName)) - + ////////////////////////////////////////////////////////////////////////////////////////////// // DateTime functions ////////////////////////////////////////////////////////////////////////////////////////////// From b38226720e9df98f9474d6643d32632491aabb23 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 17:46:14 -0700 Subject: [PATCH 30/49] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it --- .../expressions/datetimeFunctions.scala | 40 +++++++++---------- .../expressions/DatetimeFunctionsSuite.scala | 11 ++++- .../spark/sql/DatetimeExpressionsSuite.scala | 5 +++ .../execution/HiveCompatibilitySuite.scala | 10 +++-- 4 files changed, 41 insertions(+), 25 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index e162110934738..f8b0049854fc0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -322,16 +322,16 @@ case class Day(child: Expression) extends DateFormatExpression with ImplicitCast dayInYear match { case i: Int if i <= 31 => i case i: Int if i <= 59 + leap => i - 31 - case i: Int if i <= 90 + leap => i - 59 + leap - case i: Int if i <= 120 + leap => i - 90 + leap - case i: Int if i <= 151 + leap => i - 120 + leap - case i: Int if i <= 181 + leap => i - 151 + leap - case i: Int if i <= 212 + leap => i - 181 + leap - case i: Int if i <= 243 + leap => i - 212 + leap - case i: Int if i <= 273 + leap => i - 243 + leap - case i: Int if i <= 304 + leap => i - 273 + leap - case i: Int if i <= 334 + leap => i - 304 + leap - case i: Int => i - 334 + leap + case i: Int if i <= 90 + leap => i - 59 - leap + case i: Int if i <= 120 + leap => i - 90 - leap + case i: Int if i <= 151 + leap => i - 120 - leap + case i: Int if i <= 181 + leap => i - 151 - leap + case i: Int if i <= 212 + leap => i - 181 - leap + case i: Int if i <= 243 + leap => i - 212 - leap + case i: Int if i <= 273 + leap => i - 243 - leap + case i: Int if i <= 304 + leap => i - 273 - leap + case i: Int if i <= 334 + leap => i - 304 - leap + case i: Int => i - 334 - leap } } @@ -346,25 +346,25 @@ case class Day(child: Expression) extends DateFormatExpression with ImplicitCast } else if ($dayInYear <= 59 + $leap) { ${ev.primitive} = $dayInYear - 31; } else if ($dayInYear <= 90 + $leap) { - ${ev.primitive} = $dayInYear - 59 + $leap; + ${ev.primitive} = $dayInYear - 59 - $leap; } else if ($dayInYear <= 120 + $leap) { - ${ev.primitive} = $dayInYear - 90 + $leap; + ${ev.primitive} = $dayInYear - 90 - $leap; } else if ($dayInYear <= 151 + $leap) { - ${ev.primitive} = $dayInYear - 120 + $leap; + ${ev.primitive} = $dayInYear - 120 - $leap; } else if ($dayInYear <= 181 + $leap) { - ${ev.primitive} = $dayInYear - 151 + $leap; + ${ev.primitive} = $dayInYear - 151 - $leap; } else if ($dayInYear <= 212 + $leap) { - ${ev.primitive} = $dayInYear - 181 + $leap; + ${ev.primitive} = $dayInYear - 181 - $leap; } else if ($dayInYear <= 243 + $leap) { - ${ev.primitive} = $dayInYear - 212 + $leap; + ${ev.primitive} = $dayInYear - 212 - $leap; } else if ($dayInYear <= 273 + $leap) { - ${ev.primitive} = $dayInYear - 243 + $leap; + ${ev.primitive} = $dayInYear - 243 - $leap; } else if ($dayInYear <= 304 + $leap) { - ${ev.primitive} = $dayInYear - 273 + $leap; + ${ev.primitive} = $dayInYear - 273 - $leap; } else if ($dayInYear <= 334 + $leap) { - ${ev.primitive} = $dayInYear - 304 + $leap; + ${ev.primitive} = $dayInYear - 304 - $leap; } else { - ${ev.primitive} = $dayInYear - 334 + $leap; + ${ev.primitive} = $dayInYear - 334 - $leap; } """ }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index abfd2d71d77e8..90a917bdebf6a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -166,7 +166,6 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - val sdf = new SimpleDateFormat("D") (1999 to 2000).foreach { y => (0 to 11).foreach { m => (0 to 5 * 24).foreach { i => @@ -185,6 +184,16 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Day(Cast(Literal(d), DateType)), 8) checkEvaluation(Day(Cast(Literal(sdfDate.format(d)), DateType)), 8) checkEvaluation(Day(Cast(Literal(ts), DateType)), 8) + + (1999 to 2000).foreach { y => + val c = Calendar.getInstance() + c.set(y, 0, 1, 0, 0, 0) + (0 to 365).foreach { d => + c.add(Calendar.DATE, 1) + checkEvaluation(Day(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.DAY_OF_MONTH)) + } + } } test("Seconds") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index 902d4c73ad09f..bc2913076c71c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -116,6 +116,11 @@ class DatetimeExpressionsSuite extends QueryTest { checkAnswer( df.selectExpr("day(a)", "day(b)", "day(c)"), Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day(CAST(\"2008-11-01 15:32:20\" AS DATE))"), + Row(1) + ) } test("dayInYear") { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 4ada64bc21966..9f2691505bb4d 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -43,9 +43,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { override def beforeAll() { TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) - TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting - Locale.setDefault(Locale.US) + // Locale.setDefault(Locale.US) // Set a relatively small column batch size for testing purposes TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 5) // Enable in-memory partition pruning for testing purposes @@ -892,7 +892,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_lpad", "udf_ltrim", "udf_map", - "udf_minute", + // "udf_minute", we can cast dates likes '2015-03-18' to a timestamp and extract the minutes. + // Hive returns null for minute('2015-03-18') "udf_modulo", "udf_month", "udf_named_struct", @@ -919,7 +920,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_round_3", "udf_rpad", "udf_rtrim", - "udf_second", + // "udf_second", we can cast dates likes '2015-03-18' to a timestamp and extract the seconds. + // Hive returns null for second('2015-03-18') "udf_sign", "udf_sin", "udf_smallint", From d6aa14e2bdb14cb28516e264bada9b8f6e5927d7 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 21:12:04 -0700 Subject: [PATCH 31/49] [SPARK-8199] fixed Hive compatibility --- .../spark/sql/catalyst/expressions/datetimeFunctions.scala | 7 +++++++ .../sql/catalyst/expressions/DatetimeFunctionsSuite.scala | 1 + .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index f8b0049854fc0..222ffcc0ea912 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -385,6 +385,8 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa val c = ctx.freshName("cal") s""" $cal $c = $cal.getInstance(); + $c.setFirstDayOfWeek($cal.MONDAY); + $c.setMinimalDaysInFirstWeek(4); $c.setTimeInMillis($time * 1000L * 3600L * 24L); ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); """ @@ -392,6 +394,11 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa override protected def nullSafeEval(input: Any): Any = { val c = Calendar.getInstance() + + // This is set in order to be Hive and ISO-8601 compliant + c.setFirstDayOfWeek(Calendar.MONDAY) + c.setMinimalDaysInFirstWeek(4) + c.setTimeInMillis(input.asInstanceOf[Int] * 1000L * 3600L * 24L) c.get(Calendar.WEEK_OF_YEAR) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index 90a917bdebf6a..3e2446711d880 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -261,6 +261,7 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekOfYear - " + tz.getID) { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) checkEvaluation(WeekOfYear(Cast(Literal(d), DateType)), 15) checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 9f2691505bb4d..979e7f8774e3f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -43,9 +43,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { override def beforeAll() { TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) - // TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting - // Locale.setDefault(Locale.US) + Locale.setDefault(Locale.US) // Set a relatively small column batch size for testing purposes TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 5) // Enable in-memory partition pruning for testing purposes From e223bc0f92776a4d637553378d1f23fb985861f5 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 16 Jul 2015 23:52:22 -0700 Subject: [PATCH 32/49] [SPARK-8199] refactoring --- .../catalyst/analysis/FunctionRegistry.scala | 6 +- .../expressions/datetimeFunctions.scala | 322 ++++-------------- .../sql/catalyst/util/DateTimeUtils.scala | 198 ++++++++++- .../expressions/DateFunctionsSuite.scala | 259 ++++++++++++++ .../expressions/DatetimeFunctionsSuite.scala | 230 ------------- .../org/apache/spark/sql/functions.scala | 68 ++-- .../spark/sql/DateExpressionsSuite.scala | 158 +++++++++ .../spark/sql/DatetimeExpressionsSuite.scala | 135 -------- 8 files changed, 731 insertions(+), 645 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 7951c440ea01e..b747951d24a4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -179,16 +179,16 @@ object FunctionRegistry { // datetime functions expression[CurrentDate]("current_date"), expression[CurrentTimestamp]("current_timestamp"), - expression[DateFormatClass]("dateformat"), + expression[DateFormatClass]("date_format"), expression[Year]("year"), expression[Quarter]("quarter"), expression[Month]("month"), expression[Day]("day"), - expression[DayInYear]("dayinyear"), + expression[DayInYear]("day_in_year"), expression[Hour]("hour"), expression[Minute]("minute"), expression[Second]("second"), - expression[WeekOfYear]("weekofyear") + expression[WeekOfYear]("week_of_year") ) val builtin: FunctionRegistry = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 222ffcc0ea912..1963f1fb10134 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -68,16 +68,13 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu override def dataType: DataType = IntegerType override protected def nullSafeEval(timestamp: Any): Any = { - val time = timestamp.asInstanceOf[Long] / 1000 - val longTime: Long = time.asInstanceOf[Long] + TimeZone.getDefault.getOffset(time) - ((longTime / (1000 * 3600)) % 24).toInt + DateTimeUtils.getHours(timestamp.asInstanceOf[Long]) } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val tz = classOf[TimeZone].getName + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" defineCodeGen(ctx, ev, (c) => - s"""(int) ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) - / (1000 * 3600) % 24)""".stripMargin + s"""$dtu.getHours($c)""" ) } } @@ -89,16 +86,13 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn override def dataType: DataType = IntegerType override protected def nullSafeEval(timestamp: Any): Any = { - val time = timestamp.asInstanceOf[Long] / 1000 - val longTime: Long = time.asInstanceOf[Long] + TimeZone.getDefault.getOffset(time) - ((longTime / (1000 * 60)) % 60).toInt + DateTimeUtils.getMinutes(timestamp.asInstanceOf[Long]) } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val tz = classOf[TimeZone].getName + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" defineCodeGen(ctx, ev, (c) => - s"""(int) ((($c / 1000) + $tz.getDefault().getOffset($c / 1000)) - / (1000 * 60) % 60)""".stripMargin + s"""$dtu.getMinutes($c)""" ) } } @@ -109,298 +103,128 @@ case class Second(child: Expression) extends UnaryExpression with ImplicitCastIn override def dataType: DataType = IntegerType - override protected def nullSafeEval(time: Any): Any = { - (time.asInstanceOf[Long] / 1000L / 1000L % 60L).toInt + override protected def nullSafeEval(timestamp: Any): Any = { + DateTimeUtils.getSeconds(timestamp.asInstanceOf[Long]) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, (time) => { - s"""${ev.primitive} = (int) ($time / 1000L / 1000L % 60L);""" - }) + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getSeconds($c)""" + ) } } -abstract class DateFormatExpression extends UnaryExpression with ImplicitCastInputTypes { - self: Product => +case class DayInYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - val daysIn400Years: Int = 146097 - val to2001 = -11323 + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - // this is year -17999, calculation: 50 * daysIn400Year - val toYearZero = to2001 + 7304850 + override def dataType: DataType = IntegerType - protected def isLeapYear(year: Int): Boolean = { - (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) - } + override def prettyName: String = "day_in_year" - private[this] def yearBoundary(year: Int): Int = { - year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getDayInYear(date.asInstanceOf[Int]) } - private[this] def numYears(in: Int): Int = { - val year = in / 365 - if (in > yearBoundary(year)) year else year - 1 + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getDayInYear($c)""" + ) } +} - override def dataType: DataType = IntegerType - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - protected def calculateYearAndDayInYear(daysIn: Int): (Int, Int) = { - val daysNormalized = daysIn + toYearZero - val numOfQuarterCenturies = daysNormalized / daysIn400Years - val daysInThis400 = daysNormalized % daysIn400Years + 1 - val years = numYears(daysInThis400) - val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years - val dayInYear = daysInThis400 - yearBoundary(years) - (year, dayInYear) - } +case class Year(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - protected def codeGen(ctx: CodeGenContext, ev: GeneratedExpressionCode, input: String, - f: (String, String) => String): String = { - val daysIn400Years = ctx.freshName("daysIn400Years") - val to2001 = ctx.freshName("to2001") - val toYearZero = ctx.freshName("toYearZero") - val daysNormalized = ctx.freshName("daysNormalized") - val numOfQuarterCenturies = ctx.freshName("numOfQuarterCenturies") - val daysInThis400 = ctx.freshName("daysInThis400") - val years = ctx.freshName("years") - val year = ctx.freshName("year") - val dayInYear = ctx.freshName("dayInYear") - - s""" - int $daysIn400Years = 146097; - int $to2001 = -11323; - int $toYearZero = $to2001 + 7304850; - - int $daysNormalized = $input + $toYearZero; - int $numOfQuarterCenturies = $daysNormalized / $daysIn400Years; - int $daysInThis400 = $daysNormalized % $daysIn400Years + 1; - int $years = $daysInThis400 / 365; - - $years = ($daysInThis400 > $years * 365 + (($years / 4 ) - ($years / 100) + - ($years / 400))) ? $years : $years - 1; - - int $year = (2001 - 20000) + 400 * $numOfQuarterCenturies + $years; - int $dayInYear = $daysInThis400 - - ($years * 365 + (($years / 4 ) - ($years / 100) + ($years / 400))); - ${f(year, dayInYear)}; - """ - } -} + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) -case class DayInYear(child: Expression) extends DateFormatExpression { + override def dataType: DataType = IntegerType - override protected def nullSafeEval(input: Any): Any = { - calculateYearAndDayInYear(input.asInstanceOf[Int])._2 + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getYear(date.asInstanceOf[Int]) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, days => { - codeGen(ctx, ev, days, (year, dayInYear) => { - s"""${ev.primitive} = $dayInYear;""" - }) - }) + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getYear($c)""" + ) } } +case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) -case class Year(child: Expression) extends DateFormatExpression { + override def dataType: DataType = IntegerType - override protected def nullSafeEval(input: Any): Any = { - calculateYearAndDayInYear(input.asInstanceOf[Int])._1 + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getQuarter(date.asInstanceOf[Int]) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, days => { - codeGen(ctx, ev, days, (year, dayInYear) => { - s"""${ev.primitive} = $year;""" - }) - }) + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getQuarter($c)""" + ) } } -case class Quarter(child: Expression) extends DateFormatExpression { - - override protected def nullSafeEval(input: Any): Any = { - val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (isLeapYear(year)) 1 else 0 - dayInYear match { - case i: Int if i <= 90 + leap => 1 - case i: Int if i <= 181 + leap => 2 - case i: Int if i <= 273 + leap => 3 - case _ => 4 - } - } +case class Month(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, days => { - codeGen(ctx, ev, days, (year, dayInYear) => { - s""" - if ($dayInYear <= 90) { - ${ev.primitive} = 1; - } else if ($dayInYear <= 181) { - ${ev.primitive} = 2; - } else if ($dayInYear <= 273) { - ${ev.primitive} = 3; - } else { - ${ev.primitive} = 4; - } - """ - }) - }) - } -} + override def inputTypes: Seq[AbstractDataType] = Seq(DateType) + + override def dataType: DataType = IntegerType -case class Month(child: Expression) extends DateFormatExpression { - - override protected def nullSafeEval(input: Any): Any = { - val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (isLeapYear(year)) 1 else 0 - dayInYear match { - case i: Int if i <= 31 => 1 - case i: Int if i <= 59 + leap => 2 - case i: Int if i <= 90 + leap => 3 - case i: Int if i <= 120 + leap => 4 - case i: Int if i <= 151 + leap => 5 - case i: Int if i <= 181 + leap => 6 - case i: Int if i <= 212 + leap => 7 - case i: Int if i <= 243 + leap => 8 - case i: Int if i <= 273 + leap => 9 - case i: Int if i <= 304 + leap => 10 - case i: Int if i <= 334 + leap => 11 - case _ => 12 - } + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getMonth(date.asInstanceOf[Int]) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, days => { - codeGen(ctx, ev, days, (year, dayInYear) => { - val leap = ctx.freshName("leap") - s""" - int $leap = ($year % 4) == 0 && (($year % 100) != 0 || ($year % 400) == 0) ? 1 : 0; - if ($dayInYear <= 31) { - ${ev.primitive} = 1; - } else if ($dayInYear <= 59 + $leap) { - ${ev.primitive} = 2; - } else if ($dayInYear <= 90 + $leap) { - ${ev.primitive} = 3; - } else if ($dayInYear <= 120 + $leap) { - ${ev.primitive} = 4; - } else if ($dayInYear <= 151 + $leap) { - ${ev.primitive} = 5; - } else if ($dayInYear <= 181 + $leap) { - ${ev.primitive} = 6; - } else if ($dayInYear <= 212 + $leap) { - ${ev.primitive} = 7; - } else if ($dayInYear <= 243 + $leap) { - ${ev.primitive} = 8; - } else if ($dayInYear <= 273 + $leap) { - ${ev.primitive} = 9; - } else if ($dayInYear <= 304 + $leap) { - ${ev.primitive} = 10; - } else if ($dayInYear <= 334 + $leap) { - ${ev.primitive} = 11; - } else { - ${ev.primitive} = 12; - } - """ - }) - }) + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getMonth($c)""" + ) } } -case class Day(child: Expression) extends DateFormatExpression with ImplicitCastInputTypes { - - override def dataType: DataType = IntegerType +case class Day(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - override protected def nullSafeEval(input: Any): Any = { - val (year, dayInYear) = calculateYearAndDayInYear(input.asInstanceOf[Int]) - val leap = if (isLeapYear(year)) 1 else 0 - dayInYear match { - case i: Int if i <= 31 => i - case i: Int if i <= 59 + leap => i - 31 - case i: Int if i <= 90 + leap => i - 59 - leap - case i: Int if i <= 120 + leap => i - 90 - leap - case i: Int if i <= 151 + leap => i - 120 - leap - case i: Int if i <= 181 + leap => i - 151 - leap - case i: Int if i <= 212 + leap => i - 181 - leap - case i: Int if i <= 243 + leap => i - 212 - leap - case i: Int if i <= 273 + leap => i - 243 - leap - case i: Int if i <= 304 + leap => i - 273 - leap - case i: Int if i <= 334 + leap => i - 304 - leap - case i: Int => i - 334 - leap - } + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getDayOfMonth(date.asInstanceOf[Int]) } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - nullSafeCodeGen(ctx, ev, days => { - codeGen(ctx, ev, days, (year, dayInYear) => { - val leap = ctx.freshName("leap") - s""" - int $leap = ($year % 4) == 0 && (($year % 100) != 0 || ($year % 400) == 0) ? 1 : 0; - if ($dayInYear <= 31) { - ${ev.primitive} = $dayInYear; - } else if ($dayInYear <= 59 + $leap) { - ${ev.primitive} = $dayInYear - 31; - } else if ($dayInYear <= 90 + $leap) { - ${ev.primitive} = $dayInYear - 59 - $leap; - } else if ($dayInYear <= 120 + $leap) { - ${ev.primitive} = $dayInYear - 90 - $leap; - } else if ($dayInYear <= 151 + $leap) { - ${ev.primitive} = $dayInYear - 120 - $leap; - } else if ($dayInYear <= 181 + $leap) { - ${ev.primitive} = $dayInYear - 151 - $leap; - } else if ($dayInYear <= 212 + $leap) { - ${ev.primitive} = $dayInYear - 181 - $leap; - } else if ($dayInYear <= 243 + $leap) { - ${ev.primitive} = $dayInYear - 212 - $leap; - } else if ($dayInYear <= 273 + $leap) { - ${ev.primitive} = $dayInYear - 243 - $leap; - } else if ($dayInYear <= 304 + $leap) { - ${ev.primitive} = $dayInYear - 273 - $leap; - } else if ($dayInYear <= 334 + $leap) { - ${ev.primitive} = $dayInYear - 304 - $leap; - } else { - ${ev.primitive} = $dayInYear - 334 - $leap; - } - """ - }) - }) + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getDayOfMonth($c)""" + ) } - } case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - override def dataType: DataType = IntegerType - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = - nullSafeCodeGen(ctx, ev, (time) => { - val cal = classOf[Calendar].getName - val c = ctx.freshName("cal") - s""" - $cal $c = $cal.getInstance(); - $c.setFirstDayOfWeek($cal.MONDAY); - $c.setMinimalDaysInFirstWeek(4); - $c.setTimeInMillis($time * 1000L * 3600L * 24L); - ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); - """ - }) + override def dataType: DataType = IntegerType - override protected def nullSafeEval(input: Any): Any = { - val c = Calendar.getInstance() + override def prettyName: String = "week_of_year" - // This is set in order to be Hive and ISO-8601 compliant - c.setFirstDayOfWeek(Calendar.MONDAY) - c.setMinimalDaysInFirstWeek(4) + override protected def nullSafeEval(date: Any): Any = { + DateTimeUtils.getWeekOfYear(date.asInstanceOf[Int]) + } - c.setTimeInMillis(input.asInstanceOf[Int] * 1000L * 3600L * 24L) - c.get(Calendar.WEEK_OF_YEAR) + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + defineCodeGen(ctx, ev, (c) => + s"""$dtu.getWeekOfYear($c)""" + ) } } @@ -411,6 +235,8 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) + override def prettyName: String = "date_format" + override protected def nullSafeEval(date: Any, format: Any): Any = { val sdf = new SimpleDateFormat(format.toString) UTF8String.fromString(sdf.format(new Date(date.asInstanceOf[Long] / 1000))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 53c32a0a9802b..20af514bef2c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} -import java.util.{Calendar, TimeZone} +import java.util.{TimeZone, Calendar} import org.apache.spark.unsafe.types.UTF8String @@ -39,6 +39,23 @@ object DateTimeUtils { final val MICROS_PER_SECOND = 1000L * 1000L final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L + // number of days in 400 years + final val daysIn400Years: Int = 146097 + // number of days between 1.1.1970 and 1.1.2001 + final val to2001 = -11323 + + // this is year -17999, calculation: 50 * daysIn400Year + final val toYearZero = to2001 + 7304850 + + @transient lazy val defaultTimeZone = TimeZone.getDefault + @transient lazy val calendar = { + val c = Calendar.getInstance() + // This is set in order to be Hive and ISO-8601 compliant + c.setFirstDayOfWeek(Calendar.MONDAY) + c.setMinimalDaysInFirstWeek(4) + c + } + // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. private val threadLocalLocalTimeZone = new ThreadLocal[TimeZone] { @@ -378,4 +395,183 @@ object DateTimeUtils { c.set(segments(0), segments(1) - 1, segments(2), 0, 0, 0) Some((c.getTimeInMillis / 1000 / 3600 / 24).toInt) } + + /** + * Returns the hour value of a given timestamp value. The timestamp is expressed in microseconds. + */ + def getHours(timestamp: Long): Int = { + val localTs = (timestamp / 1000) + defaultTimeZone.getOffset(timestamp / 1000) + ((localTs / 1000 / 3600) % 24).toInt + } + + /** + * Returns the minute value of a given timestamp value. The timestamp is expressed in + * microseconds. + */ + def getMinutes(timestamp: Long): Int = { + val localTs = (timestamp / 1000) + defaultTimeZone.getOffset(timestamp / 1000) + ((localTs / 1000 / 60) % 60).toInt + } + + /** + * Returns the second value of a given timestamp value. The timestamp is expressed in + * microseconds. + */ + def getSeconds(timestamp: Long): Int = { + val localTs = (timestamp / 1000) + defaultTimeZone.getOffset(timestamp / 1000) + ((localTs / 1000) % 60).toInt + } + + private[this] def isLeapYear(year: Int): Boolean = { + (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) + } + + /** + * Return the number of days since the start of 400 year period. + * The second year of a 400 year period (year 1) starts on day 365. + */ + private[this] def yearBoundary(year: Int): Int = { + year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) + } + + /** + * Calculates the number of years for the given number of days. This depends + * on a 400 year period. + * @param days days since the beginning of the 400 year period + * @return number of year + */ + private[this] def numYears(days: Int): Int = { + val year = days / 365 + if (days > yearBoundary(year)) year else year - 1 + } + + /** + * Calculates the year and and the number of the day in the year for the given + * number of days. The given days is the number of days since 1.1.1970. + * + * The calculation uses the fact that the period 1.1.2001 until 31.12.2400 is + * equals to the period 1.1.1601 until 31.12.2000. + */ + private[this] def getYearAndDayInYear(daysSince1970: Int): (Int, Int) = { + // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) + val daysNormalized = daysSince1970 + toYearZero + val numOfQuarterCenturies = daysNormalized / daysIn400Years + val daysInThis400 = daysNormalized % daysIn400Years + 1 + val years = numYears(daysInThis400) + val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years + val dayInYear = daysInThis400 - yearBoundary(years) + (year, dayInYear) + } + + /** + * Returns the 'day in year' value for the given date. The date is expressed in days + * since 1.1.1970. + */ + def getDayInYear(date: Int): Int = { + getYearAndDayInYear(date)._2 + } + + /** + * Returns the year value for the given date. The date is expressed in days + * since 1.1.1970. + */ + def getYear(date: Int): Int = { + getYearAndDayInYear(date)._1 + } + + /** + * Returns the quarter for the given date. The date is expressed in days + * since 1.1.1970. + */ + def getQuarter(date: Int): Int = { + val (year, dayInYear) = getYearAndDayInYear(date) + val leap = if (isLeapYear(year)) 1 else 0 + if (dayInYear <= 90 + leap) { + 1 + } else if (dayInYear <= 181 + leap) { + 2 + } else if (dayInYear <= 273 + leap) { + 3 + } else { + 4 + } + } + + /** + * Returns the month value for the given date. The date is expressed in days + * since 1.1.1970. January is month 1. + */ + def getMonth(date: Int): Int = { + val (year, dayInYear) = getYearAndDayInYear(date) + val leap = if (isLeapYear(year)) 1 else 0 + if (dayInYear <= 31) { + 1 + } else if (dayInYear <= 59 + leap) { + 2 + } else if (dayInYear <= 90 + leap) { + 3 + } else if (dayInYear <= 120 + leap) { + 4 + } else if (dayInYear <= 151 + leap) { + 5 + } else if (dayInYear <= 181 + leap) { + 6 + } else if (dayInYear <= 212 + leap) { + 7 + } else if (dayInYear <= 243 + leap) { + 8 + } else if (dayInYear <= 273 + leap) { + 9 + } else if (dayInYear <= 304 + leap) { + 10 + } else if (dayInYear <= 334 + leap) { + 11 + } else { + 12 + } + } + + /** + * Returns the 'day of month' value for the given date. The date is expressed in days + * since 1.1.1970. + */ + def getDayOfMonth(date: Int): Int = { + val (year, dayInYear) = getYearAndDayInYear(date) + val leap = if (isLeapYear(year)) 1 else 0 + if (dayInYear <= 31) { + dayInYear + } else if (dayInYear <= 59 + leap) { + dayInYear - 31 + } else if (dayInYear <= 90 + leap) { + dayInYear - 59 - leap + } else if (dayInYear <= 120 + leap) { + dayInYear - 90 - leap + } else if (dayInYear <= 151 + leap) { + dayInYear - 120 - leap + } else if (dayInYear <= 181 + leap) { + dayInYear - 151 - leap + } else if (dayInYear <= 212 + leap) { + dayInYear - 181 - leap + } else if (dayInYear <= 243 + leap) { + dayInYear - 212 - leap + } else if (dayInYear <= 273 + leap) { + dayInYear - 243 - leap + } else if (dayInYear <= 304 + leap) { + dayInYear - 273 - leap + } else if (dayInYear <= 334 + leap) { + dayInYear - 304 - leap + } else { + dayInYear - 334 - leap + } + } + + /** + * Returns the week number for the given date. The date is expressed in days since + * 1.1.1970. The first week of the has to hav eat least 4 days. The first day of a + * week is Monday. + */ + def getWeekOfYear(date: Int): Int = { + calendar.setTimeInMillis(date * MILLIS_PER_DAY) + calendar.get(Calendar.WEEK_OF_YEAR) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala new file mode 100644 index 0000000000000..7ea09e354a159 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.sql.{Timestamp, Date} +import java.text.SimpleDateFormat +import java.util.{TimeZone, Calendar} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{StringType, TimestampType, DateType} + +class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { + + val oldDefault = TimeZone.getDefault + + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + + test("Day in Year") { + val sdfDay = new SimpleDateFormat("D") + (2002 to 2004).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (1998 to 2002).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (1969 to 1970).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (2402 to 2404).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + + (2398 to 2402).foreach { y => + (0 to 11).foreach { m => + (0 to 5).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.DATE, i) + checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + sdfDay.format(c.getTime).toInt) + } + } + } + } + + test("Year") { + checkEvaluation(Year(Literal.create(null, DateType)), null) + checkEvaluation(Year(Cast(Literal(d), DateType)), 2015) + checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType)), 2015) + checkEvaluation(Year(Cast(Literal(ts), DateType)), 2013) + + val c = Calendar.getInstance() + (2000 to 2010).foreach { y => + (0 to 11 by 11).foreach { m => + c.set(y, m, 28) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.YEAR)) + } + } + } + } + + test("Quarter") { + checkEvaluation(Quarter(Literal.create(null, DateType)), null) + checkEvaluation(Quarter(Cast(Literal(d), DateType)), 2) + checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType)), 2) + checkEvaluation(Quarter(Cast(Literal(ts), DateType)), 4) + + val c = Calendar.getInstance() + (2003 to 2004).foreach { y => + (0 to 11 by 3).foreach { m => + c.set(y, m, 28, 0, 0, 0) + (0 to 5 * 24).foreach { i => + c.add(Calendar.HOUR_OF_DAY, 1) + checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) / 3 + 1) + } + } + } + } + + test("Month") { + checkEvaluation(Month(Literal.create(null, DateType)), null) + checkEvaluation(Month(Cast(Literal(d), DateType)), 4) + checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) + checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) + + (2003 to 2004).foreach { y => + (0 to 11).foreach { m => + (0 to 5 * 24).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.HOUR_OF_DAY, i) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) + 1) + } + } + } + + (1999 to 2000).foreach { y => + (0 to 11).foreach { m => + (0 to 5 * 24).foreach { i => + val c = Calendar.getInstance() + c.set(y, m, 28, 0, 0, 0) + c.add(Calendar.HOUR_OF_DAY, i) + checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.MONTH) + 1) + } + } + } + } + + test("Day") { + checkEvaluation(Day(Literal.create(null, DateType)), null) + checkEvaluation(Day(Cast(Literal(d), DateType)), 8) + checkEvaluation(Day(Cast(Literal(sdfDate.format(d)), DateType)), 8) + checkEvaluation(Day(Cast(Literal(ts), DateType)), 8) + + (1999 to 2000).foreach { y => + val c = Calendar.getInstance() + c.set(y, 0, 1, 0, 0, 0) + (0 to 365).foreach { d => + c.add(Calendar.DATE, 1) + checkEvaluation(Day(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), + c.get(Calendar.DAY_OF_MONTH)) + } + } + } + + test("Seconds") { + checkEvaluation(Second(Literal.create(null, DateType)), null) + checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) + checkEvaluation(Second(Literal(ts)), 15) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { s => + c.set(2015, 18, 3, 3, 5, s) + checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.SECOND)) + } + } + + test("WeekOfYear") { + checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) + checkEvaluation(WeekOfYear(Cast(Literal(d), DateType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) + checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) + checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) + } + + testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) + testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) + testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) + + def testWithTimezone(tz: TimeZone) { + TimeZone.setDefault(tz) + test("DateFormat - " + tz.getID) { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } + + test("Hour - " + tz.getID) { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) + + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) + } + } + } + } + + test("Minute - " + tz.getID) { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) + + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) + } + } + } + + TimeZone.setDefault(oldDefault) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index 3e2446711d880..6187a6eb0bc06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -27,8 +27,6 @@ import org.apache.spark.sql.types.{StringType, DateType, TimestampType} class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - val oldDefault = TimeZone.getDefault - test("datetime function current_date") { val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] @@ -42,232 +40,4 @@ class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(math.abs(t1 - ct.getTime) < 5000) } - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val sdfDate = new SimpleDateFormat("yyyy-MM-dd") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - - test("Day in Year") { - val sdfDay = new SimpleDateFormat("D") - (2002 to 2004).foreach { y => - (0 to 11).foreach { m => - (0 to 5).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) - checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - sdfDay.format(c.getTime).toInt) - } - } - } - - (1998 to 2002).foreach { y => - (0 to 11).foreach { m => - (0 to 5).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) - checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - sdfDay.format(c.getTime).toInt) - } - } - } - - (1969 to 1970).foreach { y => - (0 to 11).foreach { m => - (0 to 5).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) - checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - sdfDay.format(c.getTime).toInt) - } - } - } - - (2402 to 2404).foreach { y => - (0 to 11).foreach { m => - (0 to 5).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) - checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - sdfDay.format(c.getTime).toInt) - } - } - } - - (2398 to 2402).foreach { y => - (0 to 11).foreach { m => - (0 to 5).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) - checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - sdfDay.format(c.getTime).toInt) - } - } - } - } - - test("Year") { - checkEvaluation(Year(Literal.create(null, DateType)), null) - checkEvaluation(Year(Cast(Literal(d), DateType)), 2015) - checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType)), 2015) - checkEvaluation(Year(Cast(Literal(ts), DateType)), 2013) - - val c = Calendar.getInstance() - (2000 to 2010).foreach { y => - (0 to 11 by 11).foreach { m => - c.set(y, m, 28) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Year(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - c.get(Calendar.YEAR)) - } - } - } - } - - test("Quarter") { - checkEvaluation(Quarter(Literal.create(null, DateType)), null) - checkEvaluation(Quarter(Cast(Literal(d), DateType)), 2) - checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType)), 2) - checkEvaluation(Quarter(Cast(Literal(ts), DateType)), 4) - - val c = Calendar.getInstance() - (2003 to 2004).foreach { y => - (0 to 11 by 3).foreach { m => - c.set(y, m, 28, 0, 0, 0) - (0 to 5 * 24).foreach { i => - c.add(Calendar.HOUR_OF_DAY, 1) - checkEvaluation(Quarter(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - c.get(Calendar.MONTH) / 3 + 1) - } - } - } - } - - test("Month") { - checkEvaluation(Month(Literal.create(null, DateType)), null) - checkEvaluation(Month(Cast(Literal(d), DateType)), 4) - checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) - checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) - - (2003 to 2004).foreach { y => - (0 to 11).foreach { m => - (0 to 5 * 24).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.HOUR_OF_DAY, i) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - c.get(Calendar.MONTH) + 1) - } - } - } - - (1999 to 2000).foreach { y => - (0 to 11).foreach { m => - (0 to 5 * 24).foreach { i => - val c = Calendar.getInstance() - c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.HOUR_OF_DAY, i) - checkEvaluation(Month(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - c.get(Calendar.MONTH) + 1) - } - } - } - } - - test("Day") { - checkEvaluation(Day(Literal.create(null, DateType)), null) - checkEvaluation(Day(Cast(Literal(d), DateType)), 8) - checkEvaluation(Day(Cast(Literal(sdfDate.format(d)), DateType)), 8) - checkEvaluation(Day(Cast(Literal(ts), DateType)), 8) - - (1999 to 2000).foreach { y => - val c = Calendar.getInstance() - c.set(y, 0, 1, 0, 0, 0) - (0 to 365).foreach { d => - c.add(Calendar.DATE, 1) - checkEvaluation(Day(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), - c.get(Calendar.DAY_OF_MONTH)) - } - } - } - - test("Seconds") { - checkEvaluation(Second(Literal.create(null, DateType)), null) - checkEvaluation(Second(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType)), 15) - checkEvaluation(Second(Literal(ts)), 15) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { s => - c.set(2015, 18, 3, 3, 5, s) - checkEvaluation(Second(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.SECOND)) - } - } - - testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) - testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) - testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) - - def testWithTimezone(tz: TimeZone) { - TimeZone.setDefault(tz) - test("DateFormat - " + tz.getID) { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } - - test("Hour - " + tz.getID) { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) - - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) - } - } - } - } - - test("Minute - " + tz.getID) { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) - - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) - } - } - } - - test("WeekOfYear - " + tz.getID) { - checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) - checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) - checkEvaluation(WeekOfYear(Cast(Literal(d), DateType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) - } - - TimeZone.setDefault(oldDefault) - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index dd50b99400992..906d760a03c14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1706,149 +1706,161 @@ object functions { /** * Converts a date/timestamp/string to a value of string in the format specified by the date - * format given by the second argument + * format given by the second argument. + * + * A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All + * pattern letters of [[java.text.SimpleDateFormat]] can be used. + * + * NOTE: Use when ever possible specialized functions like [[year]]. These benefit from a + * specialized implementation. * - * NOTE: Prefer using functions like year. These use an optimized implementation. * @group datetime_funcs * @since 1.5.0 */ - def dateFormat(l: Column, r: String): Column = DateFormatClass(l.expr, Literal(r)) + def date_format(dateExpr: Column, format: String): Column = + DateFormatClass(dateExpr.expr, Literal(format)) /** * Converts a date/timestamp/string to a value of string in the format specified by the date - * format given by the second argument + * format given by the second argument. + * + * A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All + * pattern letters of [[java.text.SimpleDateFormat]] can be used. + * + * NOTE: Use when ever possible specialized functions like [[year]]. These benefit from a + * specialized implementation. * - * NOTE: Prefer using functions like year. These use an optimized implementation. * @group datetime_funcs * @since 1.5.0 */ - def dateFormat(l: String, r: String): Column = dateFormat(Column(l), r) + def date_format(dateColumnName: String, format: String): Column = + date_format(Column(dateColumnName), format) /** - * Extracts the year as an integer from a given date/timestamp/string + * Extracts the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def year(e: Column): Column = Year(e.expr) /** - * Extracts the year as an integer from a given date/timestamp/string + * Extracts the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def year(columnName: String): Column = year(Column(columnName)) /** - * Extracts the quarter as an integer from a given date/timestamp/string + * Extracts the quarter as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def quarter(e: Column): Column = Quarter(e.expr) /** - * Extracts the quarter as an integer from a given date/timestamp/string + * Extracts the quarter as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def quarter(columnName: String): Column = quarter(Column(columnName)) /** - * Extracts the month as an integer from a given date/timestamp/string + * Extracts the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def month(e: Column): Column = Month(e.expr) /** - * Extracts the month as an integer from a given date/timestamp/string + * Extracts the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def month(columnName: String): Column = month(Column(columnName)) /** - * Extracts the day as an integer from a given date/timestamp/string + * Extracts the day as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def day(e: Column): Column = Day(e.expr) /** - * Extracts the day as an integer from a given date/timestamp/string + * Extracts the day as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def day(columnName: String): Column = day(Column(columnName)) /** - * Extracts the day of the year as an integer from a given date/timestamp/string + * Extracts the day of the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def dayInYear(e: Column): Column = DayInYear(e.expr) + def day_in_year(e: Column): Column = DayInYear(e.expr) /** - * Extracts the day of the year as an integer from a given date/timestamp/string + * Extracts the day of the year as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def dayInYear(columnName: String): Column = dayInYear(Column(columnName)) + def day_in_year(columnName: String): Column = day_in_year(Column(columnName)) /** - * Extracts the hours as an integer from a given date/timestamp/string + * Extracts the hours as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def hour(e: Column): Column = Hour(e.expr) /** - * Extracts the hours as an integer from a given date/timestamp/string + * Extracts the hours as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def hour(columnName: String): Column = hour(Column(columnName)) /** - * Extracts the minutes as an integer from a given date/timestamp/string + * Extracts the minutes as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def minute(e: Column): Column = Minute(e.expr) /** - * Extracts the minutes as an integer from a given date/timestamp/string + * Extracts the minutes as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def minute(columnName: String): Column = minute(Column(columnName)) /** - * Extracts the seconds as an integer from a given date/timestamp/string + * Extracts the seconds as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def second(e: Column): Column = Second(e.expr) /** - * Extracts the seconds as an integer from a given date/timestamp/string + * Extracts the seconds as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def second(columnName: String): Column = second(Column(columnName)) /** - * Extracts the week number as an integer from a given date/timestamp/string + * Extracts the week number as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def weekOfYear(e: Column): Column = WeekOfYear(e.expr) + def week_of_year(e: Column): Column = WeekOfYear(e.expr) /** - * Extracts the week number as an integer from a given date/timestamp/string + * Extracts the week number as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ - def weekOfYear(columnName: String): Column = weekOfYear(Column(columnName)) + def week_of_year(columnName: String): Column = week_of_year(Column(columnName)) /** * Formats the number X to a format like '#,###,###.##', rounded to d decimal places, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala new file mode 100644 index 0000000000000..9895124d9f33c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.sql.{Timestamp, Date} +import java.text.SimpleDateFormat + +import org.apache.spark.sql.functions._ + +class DateExpressionsSuite extends QueryTest { + private lazy val ctx = org.apache.spark.sql.test.TestSQLContext + + import ctx.implicits._ + + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) + val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) + + + test("date format") { + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(date_format("a", "y"), date_format("b", "y"), date_format("c", "y")), + Row("2015", "2015", "2013")) + + checkAnswer( + df.selectExpr("date_format(a, 'y')", "date_format(b, 'y')", "date_format(c, 'y')"), + Row("2015", "2015", "2013")) + } + + test("year") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(year("a"), year("b"), year("c")), + Row(2015, 2015, 2013)) + + checkAnswer( + df.selectExpr("year(a)", "year(b)", "year(c)"), + Row(2015, 2015, 2013)) + } + + test("quarter") { + val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(quarter("a"), quarter("b"), quarter("c")), + Row(2, 2, 4)) + + checkAnswer( + df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), + Row(2, 2, 4)) + } + + test("month") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(month("a"), month("b"), month("c")), + Row(4, 4, 4)) + + checkAnswer( + df.selectExpr("month(a)", "month(b)", "month(c)"), + Row(4, 4, 4)) + } + + test("day") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day("a"), day("b"), day("c")), + Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day(a)", "day(b)", "day(c)"), + Row(8, 8, 8)) + } + + test("day in year") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day_in_year("a"), day_in_year("b"), day_in_year("c")), + Row(98, 98, 98)) + + checkAnswer( + df.selectExpr("day_in_year(a)", "day_in_year(b)", "day_in_year(c)"), + Row(98, 98, 98)) + } + + test("hour") { + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(hour("a"), hour("b"), hour("c")), + Row(0, 13, 13)) + + checkAnswer( + df.selectExpr("hour(a)", "hour(b)", "hour(c)"), + Row(0, 13, 13)) + } + + test("minute") { + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(minute("a"), minute("b"), minute("c")), + Row(0, 10, 10)) + + checkAnswer( + df.selectExpr("minute(a)", "minute(b)", "minute(c)"), + Row(0, 10, 10)) + } + + test("second") { + val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(second("a"), second("b"), second("c")), + Row(0, 15, 15)) + + checkAnswer( + df.selectExpr("second(a)", "second(b)", "second(c)"), + Row(0, 15, 15)) + } + + test("week of year") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(week_of_year("a"), week_of_year("b"), week_of_year("c")), + Row(15, 15, 14)) + + checkAnswer( + df.selectExpr("week_of_year(a)", "week_of_year(b)", "week_of_year(c)"), + Row(15, 15, 14)) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index bc2913076c71c..4ac400ccdf1fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -47,139 +47,4 @@ class DatetimeExpressionsSuite extends QueryTest { assert(math.abs(ctx.sql("""SELECT CURRENT_TIMESTAMP()""").collect().head.getTimestamp( 0).getTime - System.currentTimeMillis()) < 5000) } - - - - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val sdfDate = new SimpleDateFormat("yyyy-MM-dd") - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) - - - test("date format") { - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(dateFormat("a", "y"), dateFormat("b", "y"), dateFormat("c", "y")), - Row("2015", "2015", "2013")) - - checkAnswer( - df.selectExpr("dateFormat(a, 'y')", "dateFormat(b, 'y')", "dateFormat(c, 'y')"), - Row("2015", "2015", "2013")) - } - - test("year") { - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(year("a"), year("b"), year("c")), - Row(2015, 2015, 2013)) - - checkAnswer( - df.selectExpr("year(a)", "year(b)", "year(c)"), - Row(2015, 2015, 2013)) - } - - test("quarter") { - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) - - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(quarter("a"), quarter("b"), quarter("c")), - Row(2, 2, 4)) - - checkAnswer( - df.selectExpr("quarter(a)", "quarter(b)", "quarter(c)"), - Row(2, 2, 4)) - } - - test("month") { - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(month("a"), month("b"), month("c")), - Row(4, 4, 4)) - - checkAnswer( - df.selectExpr("month(a)", "month(b)", "month(c)"), - Row(4, 4, 4)) - } - - test("day") { - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(day("a"), day("b"), day("c")), - Row(8, 8, 8)) - - checkAnswer( - df.selectExpr("day(a)", "day(b)", "day(c)"), - Row(8, 8, 8)) - - checkAnswer( - df.selectExpr("day(CAST(\"2008-11-01 15:32:20\" AS DATE))"), - Row(1) - ) - } - - test("dayInYear") { - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(dayInYear("a"), dayInYear("b"), dayInYear("c")), - Row(98, 98, 98)) - - checkAnswer( - df.selectExpr("dayInYear(a)", "dayInYear(b)", "dayInYear(c)"), - Row(98, 98, 98)) - } - - test("hour") { - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(hour("a"), hour("b"), hour("c")), - Row(0, 13, 13)) - - checkAnswer( - df.selectExpr("hour(a)", "hour(b)", "hour(c)"), - Row(0, 13, 13)) - } - - test("minute") { - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(minute("a"), minute("b"), minute("c")), - Row(0, 10, 10)) - - checkAnswer( - df.selectExpr("minute(a)", "minute(b)", "minute(c)"), - Row(0, 10, 10)) - } - - test("second") { - val df = Seq((d, sdf.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(second("a"), second("b"), second("c")), - Row(0, 15, 15)) - - checkAnswer( - df.selectExpr("second(a)", "second(b)", "second(c)"), - Row(0, 15, 15)) - } - - test("weekOfYear") { - val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") - - checkAnswer( - df.select(weekOfYear("a"), weekOfYear("b"), weekOfYear("c")), - Row(15, 15, 15)) - - checkAnswer( - df.selectExpr("weekOfYear(a)", "weekOfYear(b)", "weekOfYear(c)"), - Row(15, 15, 15)) - } } From 56c4a9237c531b2d682f2cd00088587c908842be Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 00:08:39 -0700 Subject: [PATCH 33/49] [SPARK-8199] update python docu --- python/pyspark/sql/functions.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 4a3744295da57..9568d0b4c6576 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -653,18 +653,24 @@ def ntile(n): @since(1.5) -def dateFormat(dateCol, formatCol): +def dateFormat(dateCol, format): """ - Convert the given date into the format specified by the second argument. - Return type is always string. - NOTE: Prefer using functions like year. These use an optimized implementation. + Converts a date/timestamp/string to a value of string in the format specified by the date + format given by the second argument. + + A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All + pattern letters of the Java class `java.text.SimpleDateFormat can be used. + + NOTE: Use when ever possible specialized functions like `year`. These benefit from a + specialized implementation. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.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)) + return Column(sc._jvm.functions.dateFormat(dateCol, format)) @since(1.5) From d01b97765001b337106c0ad2cb3ecb1a6e549395 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 00:10:24 -0700 Subject: [PATCH 34/49] [SPARK-8199] python underscore --- python/pyspark/sql/functions.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9568d0b4c6576..6c7c37fecc028 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -653,7 +653,7 @@ def ntile(n): @since(1.5) -def dateFormat(dateCol, format): +def date_format(dateCol, format): """ Converts a date/timestamp/string to a value of string in the format specified by the date format given by the second argument. @@ -666,7 +666,7 @@ def dateFormat(dateCol, format): >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - >>> df0.select(dateFormat('a', 'MM/dd/yyy').alias('date')).collect() + >>> df0.select(date_format('a', 'MM/dd/yyy').alias('date')).collect() [Row(date=u'04/08/2015')] """ sc = SparkContext._active_spark_context @@ -724,11 +724,11 @@ def day(col): @since(1.5) -def dayInYear(col): +def day_in_year(col): """ Extract the day of the year of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - >>> df0.select(dayInYear('a').alias('day')).collect() + >>> df0.select(day_in_year('a').alias('day')).collect() [Row(day=98)] """ sc = SparkContext._active_spark_context @@ -775,12 +775,12 @@ def second(col): @since(1.5) -def weekOfYear(col): +def week_of_year(col): """ Extract the week number of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - >>> df0.select(weekOfYear('a').alias('week')).collect() + >>> df0.select(week_of_year('a').alias('week')).collect() [Row(week=15)] """ sc = SparkContext._active_spark_context From 2259299a37716705aeaf4e01e3f03ba707df940a Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 00:17:53 -0700 Subject: [PATCH 35/49] [SPARK-8199] day_of_month alias --- python/pyspark/sql/functions.py | 20 +++++++++++++++---- .../catalyst/analysis/FunctionRegistry.scala | 1 + .../org/apache/spark/sql/functions.scala | 18 +++++++++++++++-- .../spark/sql/DateExpressionsSuite.scala | 12 +++++++++++ 4 files changed, 45 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6c7c37fecc028..bdde5cc70609c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -670,7 +670,7 @@ def date_format(dateCol, format): [Row(date=u'04/08/2015')] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.dateFormat(dateCol, format)) + return Column(sc._jvm.functions.date_format(dateCol, format)) @since(1.5) @@ -715,7 +715,7 @@ def month(col): @since(1.5) def day(col): """ - Extract the day of a given date as integer. + Extract the day of the month of a given date as integer. >>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(day('a').alias('day')).collect() [Row(day=8)] """ @@ -723,6 +723,18 @@ def day(col): return Column(sc._jvm.functions.day(col)) +@since(1.5) +def day_of_month(col): + """ + Extract the day of the month of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(day_of_month('a').alias('day')).collect() + [Row(day=8)] + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.day_of_month(col)) + + @since(1.5) def day_in_year(col): """ @@ -732,7 +744,7 @@ def day_in_year(col): [Row(day=98)] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.day(col)) + return Column(sc._jvm.functions.day_in_year(col)) @since(1.5) @@ -784,7 +796,7 @@ def week_of_year(col): [Row(week=15)] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.weekOfYear(col)) + return Column(sc._jvm.functions.week_of_year(col)) class UserDefinedFunction(object): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index b747951d24a4c..98b891c144a6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -184,6 +184,7 @@ object FunctionRegistry { expression[Quarter]("quarter"), expression[Month]("month"), expression[Day]("day"), + expression[Day]("day_of_month"), expression[DayInYear]("day_in_year"), expression[Hour]("hour"), expression[Minute]("minute"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 906d760a03c14..6b03398034ffa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1779,19 +1779,33 @@ object functions { def month(columnName: String): Column = month(Column(columnName)) /** - * Extracts the day as an integer from a given date/timestamp/string. + * Extracts the day of the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def day(e: Column): Column = Day(e.expr) /** - * Extracts the day as an integer from a given date/timestamp/string. + * Extracts the day of the month as an integer from a given date/timestamp/string. * @group datetime_funcs * @since 1.5.0 */ def day(columnName: String): Column = day(Column(columnName)) + /** + * Extracts the day of the month as an integer from a given date/timestamp/string. + * @group datetime_funcs + * @since 1.5.0 + */ + def day_of_month(e: Column): Column = Day(e.expr) + + /** + * Extracts the day of the month as an integer from a given date/timestamp/string. + * @group datetime_funcs + * @since 1.5.0 + */ + def day_of_month(columnName: String): Column = day_of_month(Column(columnName)) + /** * Extracts the day of the year as an integer from a given date/timestamp/string. * @group datetime_funcs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala index 9895124d9f33c..d4c50509d46b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala @@ -95,6 +95,18 @@ class DateExpressionsSuite extends QueryTest { Row(8, 8, 8)) } + test("day of month") { + val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") + + checkAnswer( + df.select(day_of_month("a"), day_of_month("b"), day_of_month("c")), + Row(8, 8, 8)) + + checkAnswer( + df.selectExpr("day_of_month(a)", "day_of_month(b)", "day_of_month(c)"), + Row(8, 8, 8)) + } + test("day in year") { val df = Seq((d, sdfDate.format(d), ts)).toDF("a", "b", "c") From 523542db2d7470deab53c12ef324851848b55df1 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 02:18:13 -0700 Subject: [PATCH 36/49] [SPARK-8199] address comments --- python/pyspark/sql/functions.py | 2 +- .../catalyst/analysis/FunctionRegistry.scala | 11 +- .../expressions/datetimeFunctions.scala | 48 +++--- .../sql/catalyst/util/DateTimeUtils.scala | 151 +++++++++++------- .../expressions/DateFunctionsSuite.scala | 1 + .../catalyst/util/DateTimeUtilsSuite.scala | 84 ++++++++++ .../execution/HiveCompatibilitySuite.scala | 11 +- 7 files changed, 218 insertions(+), 90 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index bdde5cc70609c..6dad0e422d5fa 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -659,7 +659,7 @@ def date_format(dateCol, format): format given by the second argument. A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All - pattern letters of the Java class `java.text.SimpleDateFormat can be used. + pattern letters of the Java class `java.text.SimpleDateFormat` can be used. NOTE: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 98b891c144a6d..a17bba0f4a470 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -180,16 +180,17 @@ object FunctionRegistry { expression[CurrentDate]("current_date"), expression[CurrentTimestamp]("current_timestamp"), expression[DateFormatClass]("date_format"), - expression[Year]("year"), - expression[Quarter]("quarter"), - expression[Month]("month"), expression[Day]("day"), - expression[Day]("day_of_month"), expression[DayInYear]("day_in_year"), + expression[Day]("day_of_month"), expression[Hour]("hour"), + expression[Month]("month"), expression[Minute]("minute"), + expression[Quarter]("quarter"), expression[Second]("second"), - expression[WeekOfYear]("week_of_year") + expression[WeekOfYear]("week_of_year"), + expression[Year]("year") + ) val builtin: FunctionRegistry = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 1963f1fb10134..bb525373b123f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -72,7 +72,7 @@ case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInpu } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getHours($c)""" ) @@ -90,7 +90,7 @@ case class Minute(child: Expression) extends UnaryExpression with ImplicitCastIn } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getMinutes($c)""" ) @@ -108,7 +108,7 @@ case class Second(child: Expression) extends UnaryExpression with ImplicitCastIn } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getSeconds($c)""" ) @@ -128,7 +128,7 @@ case class DayInYear(child: Expression) extends UnaryExpression with ImplicitCas } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getDayInYear($c)""" ) @@ -147,7 +147,7 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getYear($c)""" ) @@ -165,7 +165,7 @@ case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastI } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getQuarter($c)""" ) @@ -183,7 +183,7 @@ case class Month(child: Expression) extends UnaryExpression with ImplicitCastInp } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getMonth($c)""" ) @@ -201,7 +201,7 @@ case class Day(child: Expression) extends UnaryExpression with ImplicitCastInput } override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (c) => s"""$dtu.getDayOfMonth($c)""" ) @@ -217,15 +217,25 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "week_of_year" override protected def nullSafeEval(date: Any): Any = { - DateTimeUtils.getWeekOfYear(date.asInstanceOf[Int]) + val c = Calendar.getInstance() + c.setFirstDayOfWeek(Calendar.MONDAY) + c.setMinimalDaysInFirstWeek(4) + c.setTimeInMillis(date.asInstanceOf[Int] * 1000L * 3600L * 24L) + c.get(Calendar.WEEK_OF_YEAR) } - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val dtu = "org.apache.spark.sql.catalyst.util.DateTimeUtils" - defineCodeGen(ctx, ev, (c) => - s"""$dtu.getWeekOfYear($c)""" - ) - } + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = + nullSafeCodeGen(ctx, ev, (time) => { + val cal = classOf[Calendar].getName + val c = ctx.freshName("cal") + s""" + $cal $c = $cal.getInstance(); + $c.setFirstDayOfWeek($cal.MONDAY); + $c.setMinimalDaysInFirstWeek(4); + $c.setTimeInMillis($time * 1000L * 3600L * 24L); + ${ev.primitive} = $c.get($cal.WEEK_OF_YEAR); + """ + }) } case class DateFormatClass(left: Expression, right: Expression) extends BinaryExpression @@ -237,16 +247,16 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def prettyName: String = "date_format" - override protected def nullSafeEval(date: Any, format: Any): Any = { + override protected def nullSafeEval(timestamp: Any, format: Any): Any = { val sdf = new SimpleDateFormat(format.toString) - UTF8String.fromString(sdf.format(new Date(date.asInstanceOf[Long] / 1000))) + UTF8String.fromString(sdf.format(new Date(timestamp.asInstanceOf[Long] / 1000))) } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName - defineCodeGen(ctx, ev, (date, format) => { + defineCodeGen(ctx, ev, (timestamp, format) => { s"""${ctx.stringType}.fromString((new $sdf($format.toString())) - .format(new java.sql.Date($date / 1000)))""" + .format(new java.sql.Date($timestamp / 1000)))""" }) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 20af514bef2c1..236ad2e5355b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} +import java.util import java.util.{TimeZone, Calendar} import org.apache.spark.unsafe.types.UTF8String @@ -418,8 +419,7 @@ object DateTimeUtils { * microseconds. */ def getSeconds(timestamp: Long): Int = { - val localTs = (timestamp / 1000) + defaultTimeZone.getOffset(timestamp / 1000) - ((localTs / 1000) % 60).toInt + ((timestamp / 1000 / 1000) % 60).toInt } private[this] def isLeapYear(year: Int): Boolean = { @@ -438,11 +438,12 @@ object DateTimeUtils { * Calculates the number of years for the given number of days. This depends * on a 400 year period. * @param days days since the beginning of the 400 year period - * @return number of year + * @return (number of year, days in year) */ - private[this] def numYears(days: Int): Int = { + private[this] def numYears(days: Int): (Int, Int) = { val year = days / 365 - if (days > yearBoundary(year)) year else year - 1 + val boundary = yearBoundary(year) + if (days > boundary) (year, days - boundary) else (year - 1, days - yearBoundary(year - 1)) } /** @@ -457,9 +458,8 @@ object DateTimeUtils { val daysNormalized = daysSince1970 + toYearZero val numOfQuarterCenturies = daysNormalized / daysIn400Years val daysInThis400 = daysNormalized % daysIn400Years + 1 - val years = numYears(daysInThis400) + val (years, dayInYear) = numYears(daysInThis400) val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years - val dayInYear = daysInThis400 - yearBoundary(years) (year, dayInYear) } @@ -484,13 +484,15 @@ object DateTimeUtils { * since 1.1.1970. */ def getQuarter(date: Int): Int = { - val (year, dayInYear) = getYearAndDayInYear(date) - val leap = if (isLeapYear(year)) 1 else 0 - if (dayInYear <= 90 + leap) { + var (year, dayInYear) = getYearAndDayInYear(date) + if (isLeapYear(year)) { + dayInYear = dayInYear - 1 + } + if (dayInYear <= 90) { 1 - } else if (dayInYear <= 181 + leap) { + } else if (dayInYear <= 181) { 2 - } else if (dayInYear <= 273 + leap) { + } else if (dayInYear <= 273) { 3 } else { 4 @@ -504,30 +506,42 @@ object DateTimeUtils { def getMonth(date: Int): Int = { val (year, dayInYear) = getYearAndDayInYear(date) val leap = if (isLeapYear(year)) 1 else 0 - if (dayInYear <= 31) { - 1 - } else if (dayInYear <= 59 + leap) { - 2 - } else if (dayInYear <= 90 + leap) { - 3 - } else if (dayInYear <= 120 + leap) { - 4 - } else if (dayInYear <= 151 + leap) { - 5 - } else if (dayInYear <= 181 + leap) { - 6 - } else if (dayInYear <= 212 + leap) { - 7 - } else if (dayInYear <= 243 + leap) { - 8 - } else if (dayInYear <= 273 + leap) { - 9 - } else if (dayInYear <= 304 + leap) { - 10 - } else if (dayInYear <= 334 + leap) { - 11 + if (dayInYear <= 181 + leap) { + if (dayInYear <= 90 + leap) { + if (dayInYear <= 31) { + 1 + } else if (dayInYear <= 59 + leap) { + 2 + } else { + 3 + } + } else { + if (dayInYear <= 120 + leap) { + 4 + } else if (dayInYear <= 151 + leap) { + 5 + } else { + 6 + } + } } else { - 12 + if (dayInYear <= 273 + leap) { + if (dayInYear <= 212 + leap) { + 7 + } else if (dayInYear <= 243 + leap) { + 8 + } else { + 9 + } + } else { + if (dayInYear <= 304 + leap) { + 10 + } else if (dayInYear <= 334 + leap) { + 11 + } else { + 12 + } + } } } @@ -536,38 +550,53 @@ object DateTimeUtils { * since 1.1.1970. */ def getDayOfMonth(date: Int): Int = { - val (year, dayInYear) = getYearAndDayInYear(date) - val leap = if (isLeapYear(year)) 1 else 0 - if (dayInYear <= 31) { - dayInYear - } else if (dayInYear <= 59 + leap) { - dayInYear - 31 - } else if (dayInYear <= 90 + leap) { - dayInYear - 59 - leap - } else if (dayInYear <= 120 + leap) { - dayInYear - 90 - leap - } else if (dayInYear <= 151 + leap) { - dayInYear - 120 - leap - } else if (dayInYear <= 181 + leap) { - dayInYear - 151 - leap - } else if (dayInYear <= 212 + leap) { - dayInYear - 181 - leap - } else if (dayInYear <= 243 + leap) { - dayInYear - 212 - leap - } else if (dayInYear <= 273 + leap) { - dayInYear - 243 - leap - } else if (dayInYear <= 304 + leap) { - dayInYear - 273 - leap - } else if (dayInYear <= 334 + leap) { - dayInYear - 304 - leap + var (year, dayInYear) = getYearAndDayInYear(date) + val leap = if (isLeapYear(year) && dayInYear > 59) 1 else 0 + if (dayInYear >= 60) { + dayInYear = dayInYear - 1 + } + if (dayInYear <= 181 + leap) { + if (dayInYear <= 90 + leap) { + if (dayInYear <= 31) { + dayInYear + } else if (dayInYear <= 59 + leap) { + dayInYear - 31 + } else { + dayInYear - 59 - leap + } + } else { + if (dayInYear <= 120 + leap) { + dayInYear - 90 - leap + } else if (dayInYear <= 151 + leap) { + dayInYear - 120 - leap + } else { + dayInYear - 151 - leap + } + } } else { - dayInYear - 334 - leap + if (dayInYear <= 273 + leap) { + if (dayInYear <= 212 + leap) { + dayInYear - 181 - leap + } else if (dayInYear <= 243 + leap) { + dayInYear - 212 - leap + } else { + dayInYear - 243 - leap + } + } else { + if (dayInYear <= 304 + leap) { + dayInYear - 273 - leap + } else if (dayInYear <= 334 + leap) { + dayInYear - 304 - leap + } else { + dayInYear - 334 - leap + } + } } } /** * Returns the week number for the given date. The date is expressed in days since - * 1.1.1970. The first week of the has to hav eat least 4 days. The first day of a + * 1.1.1970. The first week of the has to have at least 4 days. The first day of a * week is Monday. */ def getWeekOfYear(date: Int): Int = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala index 7ea09e354a159..bd58e07428471 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala @@ -166,6 +166,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("Day") { + checkEvaluation(Day(Cast(Literal("2000-02-29"), DateType)), 29) checkEvaluation(Day(Literal.create(null, DateType)), null) checkEvaluation(Day(Cast(Literal(d), DateType)), 8) checkEvaluation(Day(Cast(Literal(sdfDate.format(d)), DateType)), 8) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index c65fcbc4d1bc1..5a8db2e0ddb1c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -304,4 +304,88 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(DateTimeUtils.stringToTimestamp( UTF8String.fromString("2015-03-18T12:03.17-1:0:0")).isEmpty) } + + test("hours") { + val c = Calendar.getInstance() + c.set(2015, 2, 18, 13, 2, 11) + assert(DateTimeUtils.getHours(c.getTimeInMillis * 1000) === 13) + c.set(2015, 12, 8, 2, 7, 9) + assert(DateTimeUtils.getHours(c.getTimeInMillis * 1000) === 2) + } + + test("minutes") { + val c = Calendar.getInstance() + c.set(2015, 2, 18, 13, 2, 11) + assert(DateTimeUtils.getMinutes(c.getTimeInMillis * 1000) === 2) + c.set(2015, 2, 8, 2, 7, 9) + assert(DateTimeUtils.getMinutes(c.getTimeInMillis * 1000) === 7) + } + + test("seconds") { + val c = Calendar.getInstance() + c.set(2015, 2, 18, 13, 2, 11) + assert(DateTimeUtils.getSeconds(c.getTimeInMillis * 1000) === 11) + c.set(2015, 2, 8, 2, 7, 9) + assert(DateTimeUtils.getSeconds(c.getTimeInMillis * 1000) === 9) + } + + test("get day in year") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getDayInYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 77) + c.set(2012, 2, 18) + assert(DateTimeUtils.getDayInYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 78) + } + + test("get year") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 2015) + c.set(2012, 2, 18) + assert(DateTimeUtils.getYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 2012) + } + + test("get quarter") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getQuarter((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 1) + c.set(2012, 11, 18) + assert(DateTimeUtils.getQuarter((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 4) + } + + test("get month") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 3) + c.set(2012, 11, 18) + assert(DateTimeUtils.getMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 12) + } + + test("get day of month") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getDayOfMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 18) + c.set(2012, 11, 24) + assert(DateTimeUtils.getDayOfMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 24) + } + + test("get week of year") { + val c = Calendar.getInstance() + c.set(2015, 2, 18) + assert(DateTimeUtils.getWeekOfYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 12) + c.set(2012, 11, 24) + assert(DateTimeUtils.getWeekOfYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === + 51) + } } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 979e7f8774e3f..2f0f24c143018 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -115,6 +115,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // This test is totally fine except that it includes wrong queries and expects errors, but error // message format in Hive and Spark SQL differ. Should workaround this later. "udf_to_unix_timestamp", + // we can cast dates likes '2015-03-18' to a timestamp and extract the seconds. + // Hive returns null for second('2015-03-18') + "udf_second", + // we can cast dates likes '2015-03-18' to a timestamp and extract the minutes. + // Hive returns null for minute('2015-03-18') + "udf_minute", + // Cant run without local map/reduce. "index_auto_update", @@ -892,8 +899,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_lpad", "udf_ltrim", "udf_map", - // "udf_minute", we can cast dates likes '2015-03-18' to a timestamp and extract the minutes. - // Hive returns null for minute('2015-03-18') "udf_modulo", "udf_month", "udf_named_struct", @@ -920,8 +925,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_round_3", "udf_rpad", "udf_rtrim", - // "udf_second", we can cast dates likes '2015-03-18' to a timestamp and extract the seconds. - // Hive returns null for second('2015-03-18') "udf_sign", "udf_sin", "udf_smallint", From 0ad6db8e118eb5d49a53417779d7be05c431f3a6 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 07:53:22 -0700 Subject: [PATCH 37/49] [SPARK-8199] minor fix --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 236ad2e5355b8..48adc14da0146 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} -import java.util import java.util.{TimeZone, Calendar} import org.apache.spark.unsafe.types.UTF8String @@ -552,9 +551,7 @@ object DateTimeUtils { def getDayOfMonth(date: Int): Int = { var (year, dayInYear) = getYearAndDayInYear(date) val leap = if (isLeapYear(year) && dayInYear > 59) 1 else 0 - if (dayInYear >= 60) { - dayInYear = dayInYear - 1 - } + if (dayInYear <= 181 + leap) { if (dayInYear <= 90 + leap) { if (dayInYear <= 31) { From 746b80a0179170f8c294596db8c620f59afb09a3 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 08:13:34 -0700 Subject: [PATCH 38/49] [SPARK-8199] build fix --- .../spark/sql/catalyst/expressions/datetimeFunctions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index bb525373b123f..906141d01f6f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -254,8 +254,9 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName + val utf8 = classOf[UTF8String].getName defineCodeGen(ctx, ev, (timestamp, format) => { - s"""${ctx.stringType}.fromString((new $sdf($format.toString())) + s"""$utf8.fromString((new $sdf($format.toString())) .format(new java.sql.Date($timestamp / 1000)))""" }) } From cdfae279ee5a2ed8d0e58154fce1c53418d21d1a Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 10:34:36 -0700 Subject: [PATCH 39/49] [SPARK-8199] cleanup & python docstring fix --- python/pyspark/sql/functions.py | 2 +- .../sql/catalyst/util/DateTimeUtils.scala | 85 +++++++++---------- .../catalyst/util/DateTimeUtilsSuite.scala | 10 --- 3 files changed, 43 insertions(+), 54 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6dad0e422d5fa..8ae5a2792afdc 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -706,7 +706,7 @@ def month(col): >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(month('a').alias('month')).collect() - [Row(month=4)] + [Row(month=4)] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.month(col)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 48adc14da0146..e448295b744c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -503,39 +503,44 @@ object DateTimeUtils { * since 1.1.1970. January is month 1. */ def getMonth(date: Int): Int = { - val (year, dayInYear) = getYearAndDayInYear(date) - val leap = if (isLeapYear(year)) 1 else 0 - if (dayInYear <= 181 + leap) { - if (dayInYear <= 90 + leap) { + var (year, dayInYear) = getYearAndDayInYear(date) + var isLeap = isLeapYear(year) + if (isLeap && dayInYear > 60) { + dayInYear = dayInYear - 1 + isLeap = false + } + + if (dayInYear <= 181) { + if (dayInYear <= 90) { if (dayInYear <= 31) { 1 - } else if (dayInYear <= 59 + leap) { + } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { 2 } else { 3 } } else { - if (dayInYear <= 120 + leap) { + if (dayInYear <= 120) { 4 - } else if (dayInYear <= 151 + leap) { + } else if (dayInYear <= 151) { 5 } else { 6 } } } else { - if (dayInYear <= 273 + leap) { - if (dayInYear <= 212 + leap) { + if (dayInYear <= 273) { + if (dayInYear <= 212) { 7 - } else if (dayInYear <= 243 + leap) { + } else if (dayInYear <= 243) { 8 } else { 9 } } else { - if (dayInYear <= 304 + leap) { + if (dayInYear <= 304) { 10 - } else if (dayInYear <= 334 + leap) { + } else if (dayInYear <= 334) { 11 } else { 12 @@ -550,54 +555,48 @@ object DateTimeUtils { */ def getDayOfMonth(date: Int): Int = { var (year, dayInYear) = getYearAndDayInYear(date) - val leap = if (isLeapYear(year) && dayInYear > 59) 1 else 0 + var isLeap = isLeapYear(year) + if (isLeap && dayInYear > 60) { + dayInYear = dayInYear - 1 + isLeap = false + } - if (dayInYear <= 181 + leap) { - if (dayInYear <= 90 + leap) { + if (dayInYear <= 181) { + if (dayInYear <= 90) { if (dayInYear <= 31) { dayInYear - } else if (dayInYear <= 59 + leap) { + } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { dayInYear - 31 } else { - dayInYear - 59 - leap + dayInYear - 59 } } else { - if (dayInYear <= 120 + leap) { - dayInYear - 90 - leap - } else if (dayInYear <= 151 + leap) { - dayInYear - 120 - leap + if (dayInYear <= 120) { + dayInYear - 90 + } else if (dayInYear <= 151) { + dayInYear - 120 } else { - dayInYear - 151 - leap + dayInYear - 151 } } } else { - if (dayInYear <= 273 + leap) { - if (dayInYear <= 212 + leap) { - dayInYear - 181 - leap - } else if (dayInYear <= 243 + leap) { - dayInYear - 212 - leap + if (dayInYear <= 273) { + if (dayInYear <= 212) { + dayInYear - 181 + } else if (dayInYear <= 243) { + dayInYear - 212 } else { - dayInYear - 243 - leap + dayInYear - 243 } } else { - if (dayInYear <= 304 + leap) { - dayInYear - 273 - leap - } else if (dayInYear <= 334 + leap) { - dayInYear - 304 - leap + if (dayInYear <= 304) { + dayInYear - 273 + } else if (dayInYear <= 334) { + dayInYear - 304 } else { - dayInYear - 334 - leap + dayInYear - 334 } } } } - - /** - * Returns the week number for the given date. The date is expressed in days since - * 1.1.1970. The first week of the has to have at least 4 days. The first day of a - * week is Monday. - */ - def getWeekOfYear(date: Int): Int = { - calendar.setTimeInMillis(date * MILLIS_PER_DAY) - calendar.get(Calendar.WEEK_OF_YEAR) - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 5a8db2e0ddb1c..a7acf959555ba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -378,14 +378,4 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(DateTimeUtils.getDayOfMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === 24) } - - test("get week of year") { - val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getWeekOfYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 12) - c.set(2012, 11, 24) - assert(DateTimeUtils.getWeekOfYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 51) - } } From fb98ba02d7485e50aab7cc387ca68b426b7058cf Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 12:10:47 -0700 Subject: [PATCH 40/49] [SPARK-8199] python docstring fix --- python/pyspark/sql/functions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8ae5a2792afdc..87f0147879162 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -766,7 +766,7 @@ def minute(col): Extract the minutes of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) - >>> dfo.select(minute('a').alias('minute')).collect() + >>> df0.select(minute('a').alias('minute')).collect() [Row(minute=8)] """ sc = SparkContext._active_spark_context From 3c6ae2e9e6c11be11763b437d77098eaa3231e6f Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 18:09:35 -0700 Subject: [PATCH 41/49] [SPARK-8199] removed binary search --- python/pyspark/sql/functions.py | 11 ++ .../sql/catalyst/util/DateTimeUtils.scala | 116 +++++++----------- 2 files changed, 57 insertions(+), 70 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 87f0147879162..7091e89afabbf 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -652,6 +652,7 @@ def ntile(n): return Column(sc._jvm.functions.ntile(int(n))) +@ignore_unicode_prefix @since(1.5) def date_format(dateCol, format): """ @@ -677,6 +678,7 @@ def date_format(dateCol, format): def year(col): """ Extract the year of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(year('a').alias('year')).collect() @@ -690,6 +692,7 @@ def year(col): def quarter(col): """ Extract the quarter of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(quarter('a').alias('quarter')).collect() @@ -703,6 +706,7 @@ def quarter(col): def month(col): """ Extract the month of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(month('a').alias('month')).collect() @@ -716,6 +720,7 @@ def month(col): def day(col): """ Extract the day of the month of a given date as integer. + >>> sqlContext.createDataFrame([('2015-04-08',)], ['a']).select(day('a').alias('day')).collect() [Row(day=8)] """ @@ -727,7 +732,9 @@ def day(col): def day_of_month(col): """ Extract the day of the month of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(day_of_month('a').alias('day')).collect() [Row(day=8)] """ @@ -740,6 +747,7 @@ def day_in_year(col): """ Extract the day of the year of a given date as integer. >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df0.select(day_in_year('a').alias('day')).collect() [Row(day=98)] """ @@ -751,6 +759,7 @@ def day_in_year(col): def hour(col): """ Extract the hours of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df0.select(hour('a').alias('hour')).collect() @@ -777,6 +786,7 @@ def minute(col): def second(col): """ Extract the seconds of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) >>> df0.select(second('a').alias('second')).collect() @@ -790,6 +800,7 @@ def second(col): def week_of_year(col): """ Extract the week number of a given date as integer. + >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) >>> df0.select(week_of_year('a').alias('week')).collect() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index e448295b744c6..435c39400dc60 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -510,42 +510,30 @@ object DateTimeUtils { isLeap = false } - if (dayInYear <= 181) { - if (dayInYear <= 90) { - if (dayInYear <= 31) { - 1 - } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { - 2 - } else { - 3 - } - } else { - if (dayInYear <= 120) { - 4 - } else if (dayInYear <= 151) { - 5 - } else { - 6 - } - } + if (dayInYear <= 31) { + 1 + } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { + 2 + } else if (dayInYear <= 90) { + 3 + } else if (dayInYear <= 120) { + 4 + } else if (dayInYear <= 151) { + 5 + } else if (dayInYear <= 181) { + 6 + } else if (dayInYear <= 212) { + 7 + } else if (dayInYear <= 243) { + 8 + } else if (dayInYear <= 273) { + 9 + } else if (dayInYear <= 304) { + 10 + } else if (dayInYear <= 334) { + 11 } else { - if (dayInYear <= 273) { - if (dayInYear <= 212) { - 7 - } else if (dayInYear <= 243) { - 8 - } else { - 9 - } - } else { - if (dayInYear <= 304) { - 10 - } else if (dayInYear <= 334) { - 11 - } else { - 12 - } - } + 12 } } @@ -561,42 +549,30 @@ object DateTimeUtils { isLeap = false } - if (dayInYear <= 181) { - if (dayInYear <= 90) { - if (dayInYear <= 31) { - dayInYear - } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { - dayInYear - 31 - } else { - dayInYear - 59 - } - } else { - if (dayInYear <= 120) { - dayInYear - 90 - } else if (dayInYear <= 151) { - dayInYear - 120 - } else { - dayInYear - 151 - } - } + if (dayInYear <= 31) { + dayInYear + } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { + dayInYear - 31 + } else if (dayInYear <= 90) { + dayInYear - 59 + } else if (dayInYear <= 120) { + dayInYear - 90 + } else if (dayInYear <= 151) { + dayInYear - 120 + } else if (dayInYear <= 181) { + dayInYear - 151 + } else if (dayInYear <= 212) { + dayInYear - 181 + } else if (dayInYear <= 243) { + dayInYear - 212 + } else if (dayInYear <= 273) { + dayInYear - 243 + } else if (dayInYear <= 304) { + dayInYear - 273 + } else if (dayInYear <= 334) { + dayInYear - 304 } else { - if (dayInYear <= 273) { - if (dayInYear <= 212) { - dayInYear - 181 - } else if (dayInYear <= 243) { - dayInYear - 212 - } else { - dayInYear - 243 - } - } else { - if (dayInYear <= 304) { - dayInYear - 273 - } else if (dayInYear <= 334) { - dayInYear - 304 - } else { - dayInYear - 334 - } - } + dayInYear - 334 } } } From ea6c110c7740507e7979981a8d34c38b11b4e150 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 18:19:16 -0700 Subject: [PATCH 42/49] [SPARK-8199] fix after merging master --- .../spark/sql/catalyst/expressions/datetimeFunctions.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 906141d01f6f8..64323aa5a33e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -254,9 +254,8 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val sdf = classOf[SimpleDateFormat].getName - val utf8 = classOf[UTF8String].getName defineCodeGen(ctx, ev, (timestamp, format) => { - s"""$utf8.fromString((new $sdf($format.toString())) + s"""UTF8String.fromString((new $sdf($format.toString())) .format(new java.sql.Date($timestamp / 1000)))""" }) } From 4afc09c88993ae7620ea1632335c1925c9b5400f Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Fri, 17 Jul 2015 18:25:08 -0700 Subject: [PATCH 43/49] [SPARK-8199] concise leap year handling --- .../sql/catalyst/util/DateTimeUtils.scala | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 6f144b762b921..bbfc87d26d852 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -505,15 +505,17 @@ object DateTimeUtils { */ def getMonth(date: Int): Int = { var (year, dayInYear) = getYearAndDayInYear(date) - var isLeap = isLeapYear(year) - if (isLeap && dayInYear > 60) { - dayInYear = dayInYear - 1 - isLeap = false + if (isLeapYear(year)) { + if (dayInYear == 60) { + return 2 + } else if (dayInYear > 60) { + dayInYear = dayInYear - 1 + } } if (dayInYear <= 31) { 1 - } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { + } else if (dayInYear <= 59) { 2 } else if (dayInYear <= 90) { 3 @@ -544,15 +546,17 @@ object DateTimeUtils { */ def getDayOfMonth(date: Int): Int = { var (year, dayInYear) = getYearAndDayInYear(date) - var isLeap = isLeapYear(year) - if (isLeap && dayInYear > 60) { - dayInYear = dayInYear - 1 - isLeap = false + if (isLeapYear(year)) { + if (dayInYear == 60) { + return 29 + } else if (dayInYear > 60) { + dayInYear = dayInYear - 1 + } } if (dayInYear <= 31) { dayInYear - } else if (dayInYear <= 59 || (isLeap && dayInYear <= 60)) { + } else if (dayInYear <= 59) { dayInYear - 31 } else if (dayInYear <= 90) { dayInYear - 59 From 6e0c78fb5747188e015d204f9744766ff9e25678 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 07:31:23 -0700 Subject: [PATCH 44/49] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 --- .../expressions/DateFunctionsSuite.scala | 77 ++++++++----------- 1 file changed, 33 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala index bd58e07428471..b25106a2ff8ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala @@ -26,8 +26,6 @@ import org.apache.spark.sql.types.{StringType, TimestampType, DateType} class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - val oldDefault = TimeZone.getDefault - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") val sdfDate = new SimpleDateFormat("yyyy-MM-dd") val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) @@ -204,57 +202,48 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) } + + test("DateFormat") { + checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal.create(null, StringType)), null) + checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), + Literal("y")), "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") + } - testWithTimezone(TimeZone.getTimeZone("GMT-14:00")) - testWithTimezone(TimeZone.getTimeZone("GMT+04:30")) - testWithTimezone(TimeZone.getTimeZone("GMT+12:00")) - - def testWithTimezone(tz: TimeZone) { - TimeZone.setDefault(tz) - test("DateFormat - " + tz.getID) { - checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal.create(null, StringType)), null) - checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), - Literal("y")), "2015") - checkEvaluation(DateFormatClass(Literal(ts), Literal("y")), "2013") - } - - test("Hour - " + tz.getID) { - checkEvaluation(Hour(Literal.create(null, DateType)), null) - checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) - checkEvaluation(Hour(Literal(ts)), 13) + test("Hour") { + checkEvaluation(Hour(Literal.create(null, DateType)), null) + checkEvaluation(Hour(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType)), 13) + checkEvaluation(Hour(Literal(ts)), 13) - val c = Calendar.getInstance() - (0 to 24).foreach { h => - (0 to 60 by 15).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, h, m, s) - checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.HOUR_OF_DAY)) - } + val c = Calendar.getInstance() + (0 to 24).foreach { h => + (0 to 60 by 15).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, h, m, s) + checkEvaluation(Hour(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.HOUR_OF_DAY)) } } } + } - test("Minute - " + tz.getID) { - checkEvaluation(Minute(Literal.create(null, DateType)), null) - checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) - checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) - checkEvaluation(Minute(Literal(ts)), 10) + test("Minute") { + checkEvaluation(Minute(Literal.create(null, DateType)), null) + checkEvaluation(Minute(Cast(Literal(d), TimestampType)), 0) + checkEvaluation(Minute(Cast(Literal(sdf.format(d)), TimestampType)), 10) + checkEvaluation(Minute(Literal(ts)), 10) - val c = Calendar.getInstance() - (0 to 60 by 5).foreach { m => - (0 to 60 by 15).foreach { s => - c.set(2015, 18, 3, 3, m, s) - checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), - c.get(Calendar.MINUTE)) - } + val c = Calendar.getInstance() + (0 to 60 by 5).foreach { m => + (0 to 60 by 15).foreach { s => + c.set(2015, 18, 3, 3, m, s) + checkEvaluation(Minute(Cast(Literal(new Timestamp(c.getTimeInMillis)), TimestampType)), + c.get(Calendar.MINUTE)) } } - - TimeZone.setDefault(oldDefault) } } From 5983dcc7f56373bde461e57f8ebc3d8f6d5bbff0 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 07:39:46 -0700 Subject: [PATCH 45/49] [SPARK-8199] whitespace fix --- .../spark/sql/catalyst/expressions/DateFunctionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala index b25106a2ff8ea..a0991ec998311 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala @@ -202,7 +202,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) } - + test("DateFormat") { checkEvaluation(DateFormatClass(Literal.create(null, TimestampType), Literal("y")), null) checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType), From 256c357f75def1a2d1e737d475c126472613b1d2 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 10:55:30 -0700 Subject: [PATCH 46/49] [SPARK-8199] code cleanup --- .../apache/spark/sql/catalyst/util/DateTimeUtils.scala | 8 -------- .../sql/catalyst/expressions/DatetimeFunctionsSuite.scala | 6 ------ .../org/apache/spark/sql/DatetimeExpressionsSuite.scala | 4 +--- 3 files changed, 1 insertion(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index bbfc87d26d852..2cfc15bc002c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -48,14 +48,6 @@ object DateTimeUtils { final val toYearZero = to2001 + 7304850 @transient lazy val defaultTimeZone = TimeZone.getDefault - @transient lazy val calendar = { - val c = Calendar.getInstance() - // This is set in order to be Hive and ISO-8601 compliant - c.setFirstDayOfWeek(Calendar.MONDAY) - c.setMinimalDaysInFirstWeek(4) - c - } - // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. private val threadLocalLocalTimeZone = new ThreadLocal[TimeZone] { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala index 6187a6eb0bc06..1618c24871c60 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DatetimeFunctionsSuite.scala @@ -17,16 +17,10 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.{Date, Timestamp} -import java.text.SimpleDateFormat -import java.util.{TimeZone, Calendar} - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.types.{StringType, DateType, TimestampType} class DatetimeFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - test("datetime function current_date") { val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis()) val cd = CurrentDate().eval(EmptyRow).asInstanceOf[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala index 4ac400ccdf1fc..44b915304533c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatetimeExpressionsSuite.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql -import java.sql.{Timestamp, Date} -import java.text.SimpleDateFormat - import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ @@ -47,4 +44,5 @@ class DatetimeExpressionsSuite extends QueryTest { assert(math.abs(ctx.sql("""SELECT CURRENT_TIMESTAMP()""").collect().head.getTimestamp( 0).getTime - System.currentTimeMillis()) < 5000) } + } From 3e095ba4e68369f790d11c8ba624304f0c264e87 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 13:21:30 -0700 Subject: [PATCH 47/49] [SPARK-8199] style and timezone fix --- python/pyspark/sql/functions.py | 48 ++++++++----------- .../expressions/datetimeFunctions.scala | 4 +- 2 files changed, 22 insertions(+), 30 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 7091e89afabbf..0aca3788922aa 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -665,9 +665,8 @@ def date_format(dateCol, format): NOTE: Use when ever possible specialized functions like `year`. These benefit from a specialized implementation. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(date_format('a', 'MM/dd/yyy').alias('date')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(date_format('a', 'MM/dd/yyy').alias('date')).collect() [Row(date=u'04/08/2015')] """ sc = SparkContext._active_spark_context @@ -679,9 +678,8 @@ def year(col): """ Extract the year of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(year('a').alias('year')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(year('a').alias('year')).collect() [Row(year=2015)] """ sc = SparkContext._active_spark_context @@ -693,9 +691,8 @@ def quarter(col): """ Extract the quarter of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(quarter('a').alias('quarter')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(quarter('a').alias('quarter')).collect() [Row(quarter=2)] """ sc = SparkContext._active_spark_context @@ -707,9 +704,8 @@ def month(col): """ Extract the month of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(month('a').alias('month')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(month('a').alias('month')).collect() [Row(month=4)] """ sc = SparkContext._active_spark_context @@ -733,9 +729,8 @@ def day_of_month(col): """ Extract the day of the month of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(day_of_month('a').alias('day')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(day_of_month('a').alias('day')).collect() [Row(day=8)] """ sc = SparkContext._active_spark_context @@ -746,9 +741,9 @@ def day_of_month(col): def day_in_year(col): """ Extract the day of the year of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - >>> df0.select(day_in_year('a').alias('day')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(day_in_year('a').alias('day')).collect() [Row(day=98)] """ sc = SparkContext._active_spark_context @@ -760,9 +755,8 @@ def hour(col): """ Extract the hours of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) - - >>> df0.select(hour('a').alias('hour')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df.select(hour('a').alias('hour')).collect() [Row(hour=13)] """ sc = SparkContext._active_spark_context @@ -773,9 +767,9 @@ def hour(col): def minute(col): """ Extract the minutes of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) - >>> df0.select(minute('a').alias('minute')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df.select(minute('a').alias('minute')).collect() [Row(minute=8)] """ sc = SparkContext._active_spark_context @@ -787,9 +781,8 @@ def second(col): """ Extract the seconds of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) - - >>> df0.select(second('a').alias('second')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08 13:08:15',)], ['a']) + >>> df.select(second('a').alias('second')).collect() [Row(second=15)] """ sc = SparkContext._active_spark_context @@ -801,9 +794,8 @@ def week_of_year(col): """ Extract the week number of a given date as integer. - >>> df0 = sqlContext.createDataFrame([('2015-04-08',)], ['a']) - - >>> df0.select(week_of_year('a').alias('week')).collect() + >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a']) + >>> df.select(week_of_year('a').alias('week')).collect() [Row(week=15)] """ sc = SparkContext._active_spark_context diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index 64323aa5a33e6..2478eb28d6946 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -217,7 +217,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "week_of_year" override protected def nullSafeEval(date: Any): Any = { - val c = Calendar.getInstance() + val c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) c.setFirstDayOfWeek(Calendar.MONDAY) c.setMinimalDaysInFirstWeek(4) c.setTimeInMillis(date.asInstanceOf[Int] * 1000L * 3600L * 24L) @@ -229,7 +229,7 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa val cal = classOf[Calendar].getName val c = ctx.freshName("cal") s""" - $cal $c = $cal.getInstance(); + $cal $c = $cal.getInstance(java.util.TimeZone.getTimeZone("UTC")); $c.setFirstDayOfWeek($cal.MONDAY); $c.setMinimalDaysInFirstWeek(4); $c.setTimeInMillis($time * 1000L * 3600L * 24L); From bb567b6cdaf73ba24c49efbb51bb14318fee8eaa Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 16:15:47 -0700 Subject: [PATCH 48/49] [SPARK-8199] fixed test --- .../scala/org/apache/spark/sql/DateExpressionsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala index d4c50509d46b0..d24e3ee1dd8f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateExpressionsSuite.scala @@ -160,11 +160,11 @@ class DateExpressionsSuite extends QueryTest { checkAnswer( df.select(week_of_year("a"), week_of_year("b"), week_of_year("c")), - Row(15, 15, 14)) + Row(15, 15, 15)) checkAnswer( df.selectExpr("week_of_year(a)", "week_of_year(b)", "week_of_year(c)"), - Row(15, 15, 14)) + Row(15, 15, 15)) } } From f7b4c8ca3b4258fffb83db79571f8cefe16883a7 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 18 Jul 2015 18:27:15 -0700 Subject: [PATCH 49/49] [SPARK-8199] fixed bug in tests --- .../expressions/DateFunctionsSuite.scala | 8 +- .../catalyst/util/DateTimeUtilsSuite.scala | 77 ++++++------------- 2 files changed, 29 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala index a0991ec998311..49d0b0aceac0d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateFunctionsSuite.scala @@ -50,7 +50,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { (0 to 5).foreach { i => val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) + c.add(Calendar.DATE, 1) checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), sdfDay.format(c.getTime).toInt) } @@ -62,7 +62,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { (0 to 5).foreach { i => val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) + c.add(Calendar.DATE, 1) checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), sdfDay.format(c.getTime).toInt) } @@ -74,7 +74,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { (0 to 5).foreach { i => val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) + c.add(Calendar.DATE, 1) checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), sdfDay.format(c.getTime).toInt) } @@ -86,7 +86,7 @@ class DateFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { (0 to 5).foreach { i => val c = Calendar.getInstance() c.set(y, m, 28, 0, 0, 0) - c.add(Calendar.DATE, i) + c.add(Calendar.DATE, 1) checkEvaluation(DayInYear(Cast(Literal(new Date(c.getTimeInMillis)), DateType)), sdfDay.format(c.getTime).toInt) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 7d37c9a583475..1b3f978167494 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -26,6 +26,11 @@ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite { + private[this] def getInUTCDays(timestamp: Long): Int = { + val tz = TimeZone.getDefault + ((timestamp + tz.getOffset(timestamp)) / DateTimeUtils.MILLIS_PER_DAY).toInt + } + test("timestamp and us") { val now = new Timestamp(System.currentTimeMillis()) now.setNanos(1000) @@ -276,28 +281,6 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(DateTimeUtils.stringToTimestamp( UTF8String.fromString("2011-05-06 07:08:09.1000")).get === c.getTimeInMillis * 1000) - val defaultTimeZone = TimeZone.getDefault - TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) - - c = Calendar.getInstance() - c.set(2015, 2, 8, 2, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(DateTimeUtils.stringToTimestamp( - UTF8String.fromString("2015-3-8 2:0:0")).get === c.getTimeInMillis * 1000) - c.add(Calendar.MINUTE, 30) - assert(DateTimeUtils.stringToTimestamp( - UTF8String.fromString("2015-3-8 3:30:0")).get === c.getTimeInMillis * 1000) - assert(DateTimeUtils.stringToTimestamp( - UTF8String.fromString("2015-3-8 2:30:0")).get === c.getTimeInMillis * 1000) - - c = Calendar.getInstance() - c.set(2015, 10, 1, 1, 59, 0) - c.set(Calendar.MILLISECOND, 0) - c.add(Calendar.MINUTE, 31) - assert(DateTimeUtils.stringToTimestamp( - UTF8String.fromString("2015-11-1 2:30:0")).get === c.getTimeInMillis * 1000) - TimeZone.setDefault(defaultTimeZone) - assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("238")).isEmpty) assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18 123142")).isEmpty) assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T123123")).isEmpty) @@ -340,51 +323,41 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("get day in year") { val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getDayInYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 77) - c.set(2012, 2, 18) - assert(DateTimeUtils.getDayInYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 78) + c.set(2015, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getDayInYear(getInUTCDays(c.getTimeInMillis)) === 77) + c.set(2012, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78) } test("get year") { val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 2015) - c.set(2012, 2, 18) - assert(DateTimeUtils.getYear((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 2012) + c.set(2015, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getYear(getInUTCDays(c.getTimeInMillis)) === 2015) + c.set(2012, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getYear(getInUTCDays(c.getTimeInMillis)) === 2012) } test("get quarter") { val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getQuarter((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 1) - c.set(2012, 11, 18) - assert(DateTimeUtils.getQuarter((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 4) + c.set(2015, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getQuarter(getInUTCDays(c.getTimeInMillis)) === 1) + c.set(2012, 11, 18, 0, 0, 0) + assert(DateTimeUtils.getQuarter(getInUTCDays(c.getTimeInMillis)) === 4) } test("get month") { val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 3) - c.set(2012, 11, 18) - assert(DateTimeUtils.getMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 12) + c.set(2015, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getMonth(getInUTCDays(c.getTimeInMillis)) === 3) + c.set(2012, 11, 18, 0, 0, 0) + assert(DateTimeUtils.getMonth(getInUTCDays(c.getTimeInMillis)) === 12) } test("get day of month") { val c = Calendar.getInstance() - c.set(2015, 2, 18) - assert(DateTimeUtils.getDayOfMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 18) - c.set(2012, 11, 24) - assert(DateTimeUtils.getDayOfMonth((c.getTimeInMillis / DateTimeUtils.MILLIS_PER_DAY).toInt) === - 24) + c.set(2015, 2, 18, 0, 0, 0) + assert(DateTimeUtils.getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 18) + c.set(2012, 11, 24, 0, 0, 0) + assert(DateTimeUtils.getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 24) } }