From b75a80f26f0517738589c5c0f53948e36859a515 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 21:11:26 +0300 Subject: [PATCH 01/42] Add Millennium --- .../expressions/intervalExpressions.scala | 73 +++++++++++++++++++ .../sql/catalyst/util/IntervalUtils.scala | 29 ++++++++ .../interval/IntervalExpressionsSuite.scala | 42 +++++++++++ 3 files changed, 144 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala new file mode 100644 index 000000000000..fc137c23b980 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -0,0 +1,73 @@ +/* + * 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.interval + +import java.util.Locale + +import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.util.IntervalUtils +import org.apache.spark.sql.types.{AbstractDataType, CalendarIntervalType, DataType, IntegerType} +import org.apache.spark.unsafe.types.CalendarInterval + +case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getMillennium(date.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getMillennium($c)") + } +} + +object IntervalPart { + + def parseExtractField( + extractField: String, + source: Expression, + errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) +// case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) +// case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) +// case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) +// case "ISOYEAR" => IsoYear(source) +// case "QUARTER" | "QTR" => Quarter(source) +// case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) +// case "WEEK" | "W" | "WEEKS" => WeekOfYear(source) +// case "DAY" | "D" | "DAYS" => DayOfMonth(source) +// case "DAYOFWEEK" => DayOfWeek(source) +// case "DOW" => Subtract(DayOfWeek(source), Literal(1)) +// case "ISODOW" => Add(WeekDay(source), Literal(1)) +// case "DOY" => DayOfYear(source) +// case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) +// case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) +// case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) +// case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => +// Milliseconds(source) +// case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => +// Microseconds(source) +// case "EPOCH" => Epoch(source) +// case _ => errorHandleFunc + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala new file mode 100644 index 000000000000..8411b41d0d5b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -0,0 +1,29 @@ +/* + * 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.util + +import org.apache.spark.unsafe.types.CalendarInterval + +object IntervalUtils { + val MONTHS_PER_YEAR: Int = 12 + val YEARS_PER_MILLENNIUM = 1000 + + def getMillennium(interval: CalendarInterval): Int = { + (interval.months / MONTHS_PER_YEAR) / YEARS_PER_MILLENNIUM + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala new file mode 100644 index 000000000000..26b56c0361b9 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -0,0 +1,42 @@ +/* + * 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.interval + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.unsafe.types.CalendarInterval + +class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def interval(s: String): Literal = { + Literal(CalendarInterval.fromString("interval " + s)) + } + test("millennium") { + checkEvaluation(Millennium("0 years"), 0) + checkEvaluation(Millennium("9999 years"), 9) + + checkEvaluation(Millennium("1000 years"), 1) + checkEvaluation(Millennium("-2000 years"), -2) + + // Microseconds part must not be taken into account + checkEvaluation(Millennium("999 years 400 days"), 0) + + // Millennium must be taken from years and months + checkEvaluation(Millennium("999 years 12 months"), 1) + checkEvaluation(Millennium("1000 years -1 months"), 0) + } +} From b233c90d2e5e8731bab192e275abb36ba9a243d0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 21:39:31 +0300 Subject: [PATCH 02/42] Add Century --- .../expressions/intervalExpressions.scala | 18 +++++++++++++++++- .../sql/catalyst/util/IntervalUtils.scala | 7 ++++++- .../interval/IntervalExpressionsSuite.scala | 16 ++++++++++++++++ 3 files changed, 39 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index fc137c23b980..fd41ff072346 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -41,6 +41,22 @@ case class Millennium(child: Expression) extends UnaryExpression with ImplicitCa } } +case class Century(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getCentury(date.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getCentury($c)") + } +} + object IntervalPart { def parseExtractField( @@ -48,7 +64,7 @@ object IntervalPart { source: Expression, errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) -// case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) // case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) // case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) // case "ISOYEAR" => IsoYear(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 8411b41d0d5b..f0bbaafc90fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -21,9 +21,14 @@ import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { val MONTHS_PER_YEAR: Int = 12 - val YEARS_PER_MILLENNIUM = 1000 + val YEARS_PER_MILLENNIUM: Int = 1000 + val YEARS_PER_CENTURY: Int = 100 def getMillennium(interval: CalendarInterval): Int = { (interval.months / MONTHS_PER_YEAR) / YEARS_PER_MILLENNIUM } + + def getCentury(interval: CalendarInterval): Int = { + (interval.months / MONTHS_PER_YEAR) / YEARS_PER_CENTURY + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 26b56c0361b9..f970cb49c8ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -25,6 +25,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper implicit def interval(s: String): Literal = { Literal(CalendarInterval.fromString("interval " + s)) } + test("millennium") { checkEvaluation(Millennium("0 years"), 0) checkEvaluation(Millennium("9999 years"), 9) @@ -39,4 +40,19 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Millennium("999 years 12 months"), 1) checkEvaluation(Millennium("1000 years -1 months"), 0) } + + test("century") { + checkEvaluation(Century("0 years"), 0) + checkEvaluation(Century("9999 years"), 99) + + checkEvaluation(Century("1000 years"), 10) + checkEvaluation(Century("-2000 years"), -20) + + // Microseconds part must not be taken into account + checkEvaluation(Century("99 years 400 days"), 0) + + // Century must be taken from years and months + checkEvaluation(Century("99 years 12 months"), 1) + checkEvaluation(Century("100 years -1 months"), 0) + } } From 6f017a655877e13ada73ce6a10cc300c46eeb18b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 22:15:17 +0300 Subject: [PATCH 03/42] Add Decade --- .../expressions/intervalExpressions.scala | 18 +++++++++++++++++- .../sql/catalyst/util/IntervalUtils.scala | 5 +++++ .../interval/IntervalExpressionsSuite.scala | 15 +++++++++++++++ 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index fd41ff072346..a98bdebc32f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -57,6 +57,22 @@ case class Century(child: Expression) extends UnaryExpression with ImplicitCastI } } +case class Decade(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getDecade(date.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getDecade($c)") + } +} + object IntervalPart { def parseExtractField( @@ -65,7 +81,7 @@ object IntervalPart { errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) -// case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) // case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) // case "ISOYEAR" => IsoYear(source) // case "QUARTER" | "QTR" => Quarter(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index f0bbaafc90fd..9be7bf3eeceb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -23,6 +23,7 @@ object IntervalUtils { val MONTHS_PER_YEAR: Int = 12 val YEARS_PER_MILLENNIUM: Int = 1000 val YEARS_PER_CENTURY: Int = 100 + val YEARS_PER_DECADE: Int = 10 def getMillennium(interval: CalendarInterval): Int = { (interval.months / MONTHS_PER_YEAR) / YEARS_PER_MILLENNIUM @@ -31,4 +32,8 @@ object IntervalUtils { def getCentury(interval: CalendarInterval): Int = { (interval.months / MONTHS_PER_YEAR) / YEARS_PER_CENTURY } + + def getDecade(interval: CalendarInterval): Int = { + (interval.months / MONTHS_PER_YEAR) / YEARS_PER_DECADE + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index f970cb49c8ea..7923526318d5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -55,4 +55,19 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Century("99 years 12 months"), 1) checkEvaluation(Century("100 years -1 months"), 0) } + + test("decade") { + checkEvaluation(Decade("0 years"), 0) + checkEvaluation(Decade("9999 years"), 999) + + checkEvaluation(Decade("1000 years"), 100) + checkEvaluation(Decade("-2000 years"), -200) + + // Microseconds part must not be taken into account + checkEvaluation(Decade("9 years 400 days"), 0) + + // Decade must be taken from years and months + checkEvaluation(Decade("9 years 12 months"), 1) + checkEvaluation(Decade("10 years -1 months"), 0) + } } From ac1c3a81b13c590c4281c004f218347dac0e9018 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 22:20:09 +0300 Subject: [PATCH 04/42] Add Year --- .../expressions/intervalExpressions.scala | 18 +++++++++++++++++- .../sql/catalyst/util/IntervalUtils.scala | 10 +++++++--- .../interval/IntervalExpressionsSuite.scala | 15 +++++++++++++++ 3 files changed, 39 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index a98bdebc32f8..289e893b403a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -73,6 +73,22 @@ case class Decade(child: Expression) extends UnaryExpression with ImplicitCastIn } } +case class Year(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getYear(date.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getYear($c)") + } +} + object IntervalPart { def parseExtractField( @@ -82,7 +98,7 @@ object IntervalPart { case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) -// case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) // case "ISOYEAR" => IsoYear(source) // case "QUARTER" | "QTR" => Quarter(source) // case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 9be7bf3eeceb..5c3ebf3b9d2c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -25,15 +25,19 @@ object IntervalUtils { val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 + def getYear(interval: CalendarInterval): Int = { + interval.months / MONTHS_PER_YEAR + } + def getMillennium(interval: CalendarInterval): Int = { - (interval.months / MONTHS_PER_YEAR) / YEARS_PER_MILLENNIUM + getYear(interval) / YEARS_PER_MILLENNIUM } def getCentury(interval: CalendarInterval): Int = { - (interval.months / MONTHS_PER_YEAR) / YEARS_PER_CENTURY + getYear(interval) / YEARS_PER_CENTURY } def getDecade(interval: CalendarInterval): Int = { - (interval.months / MONTHS_PER_YEAR) / YEARS_PER_DECADE + getYear(interval) / YEARS_PER_DECADE } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 7923526318d5..36fa1ebca745 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -70,4 +70,19 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Decade("9 years 12 months"), 1) checkEvaluation(Decade("10 years -1 months"), 0) } + + test("year") { + checkEvaluation(Year("0 years"), 0) + checkEvaluation(Year("9999 years"), 9999) + + checkEvaluation(Year("1000 years"), 1000) + checkEvaluation(Year("-2000 years"), -2000) + + // Microseconds part must not be taken into account + checkEvaluation(Year("9 years 400 days"), 9) + + // Year must be taken from years and months + checkEvaluation(Year("9 years 12 months"), 10) + checkEvaluation(Year("10 years -1 months"), 9) + } } From 7265508b0ea9c06301c34876b6c8a8e2dd400e07 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 23:32:24 +0300 Subject: [PATCH 05/42] Add Quarter --- .../expressions/intervalExpressions.scala | 20 +++++++++++++++++-- .../sql/catalyst/util/IntervalUtils.scala | 5 +++++ .../interval/IntervalExpressionsSuite.scala | 13 ++++++++++++ 3 files changed, 36 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 289e893b403a..3a92d243abd6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -89,6 +89,22 @@ case class Year(child: Expression) extends UnaryExpression with ImplicitCastInpu } } +case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + + override def dataType: DataType = IntegerType + + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getQuarter(date.asInstanceOf[CalendarInterval]) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getQuarter($c)") + } +} + object IntervalPart { def parseExtractField( @@ -99,8 +115,8 @@ object IntervalPart { case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) -// case "ISOYEAR" => IsoYear(source) -// case "QUARTER" | "QTR" => Quarter(source) + case "ISOYEAR" => errorHandleFunc + case "QUARTER" | "QTR" => Quarter(source) // case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) // case "WEEK" | "W" | "WEEKS" => WeekOfYear(source) // case "DAY" | "D" | "DAYS" => DayOfMonth(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 5c3ebf3b9d2c..5761adb16181 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -24,6 +24,7 @@ object IntervalUtils { val YEARS_PER_MILLENNIUM: Int = 1000 val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 + val MONTHS_PER_QUARTER: Int = 3 def getYear(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -40,4 +41,8 @@ object IntervalUtils { def getDecade(interval: CalendarInterval): Int = { getYear(interval) / YEARS_PER_DECADE } + + def getQuarter(interval: CalendarInterval): Int = { + (interval.months % MONTHS_PER_YEAR) / MONTHS_PER_QUARTER + 1 + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 36fa1ebca745..04031045a675 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -85,4 +85,17 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Year("9 years 12 months"), 10) checkEvaluation(Year("10 years -1 months"), 9) } + + test("quarter") { + checkEvaluation(Quarter("0 months"), 1) + checkEvaluation(Quarter("1 months"), 1) + checkEvaluation(Quarter("-1 months"), 1) + checkEvaluation(Quarter("2 months"), 1) + checkEvaluation(Quarter("-2 months"), 1) + checkEvaluation(Quarter("1 years -1 months"), 4) + checkEvaluation(Quarter("-1 years 1 months"), -2) + checkEvaluation(Quarter("2 years 3 months"), 2) + checkEvaluation(Quarter("-2 years -3 months"), 0) + checkEvaluation(Quarter("9999 years"), 1) + } } From 554df7194fd46c2d98455beaecfb660182b34aca Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 29 Sep 2019 23:34:25 +0300 Subject: [PATCH 06/42] Move MONTHS_PER_QUARTER up --- .../org/apache/spark/sql/catalyst/util/IntervalUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 5761adb16181..2fdec9fef615 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -21,10 +21,10 @@ import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { val MONTHS_PER_YEAR: Int = 12 + val MONTHS_PER_QUARTER: Int = 3 val YEARS_PER_MILLENNIUM: Int = 1000 val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 - val MONTHS_PER_QUARTER: Int = 3 def getYear(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR From d0f89f41a6543659e533eba22f599e3f7c7dc0f5 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:05:38 +0300 Subject: [PATCH 07/42] Eliminate a warning --- .../expressions/interval/IntervalExpressionsSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 04031045a675..f3155a0602fe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.interval +import scala.language.implicitConversions + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.unsafe.types.CalendarInterval From 8174fd548c5d03be9bdd66f1e89a69227859eb93 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:06:06 +0300 Subject: [PATCH 08/42] Extend with ExpectsInputTypes --- .../expressions/intervalExpressions.scala | 32 ++++--------------- 1 file changed, 6 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 3a92d243abd6..19e3aedbd6fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -19,86 +19,66 @@ package org.apache.spark.sql.catalyst.expressions.interval import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types.{AbstractDataType, CalendarIntervalType, DataType, IntegerType} import org.apache.spark.unsafe.types.CalendarInterval -case class Millennium(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - +case class Millennium(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getMillennium(date.asInstanceOf[CalendarInterval]) } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.getMillennium($c)") } } -case class Century(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - +case class Century(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getCentury(date.asInstanceOf[CalendarInterval]) } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.getCentury($c)") } } -case class Decade(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - +case class Decade(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getDecade(date.asInstanceOf[CalendarInterval]) } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.getDecade($c)") } } -case class Year(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - +case class Year(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getYear(date.asInstanceOf[CalendarInterval]) } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.getYear($c)") } } -case class Quarter(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - +case class Quarter(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getQuarter(date.asInstanceOf[CalendarInterval]) } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.getQuarter($c)") From 6378b954e870170fba77a9a06a88e861baa80d89 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:07:04 +0300 Subject: [PATCH 09/42] Remove blank lines --- .../interval/IntervalExpressionsSuite.scala | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index f3155a0602fe..39f33a36a656 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -31,13 +31,10 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper test("millennium") { checkEvaluation(Millennium("0 years"), 0) checkEvaluation(Millennium("9999 years"), 9) - checkEvaluation(Millennium("1000 years"), 1) checkEvaluation(Millennium("-2000 years"), -2) - // Microseconds part must not be taken into account checkEvaluation(Millennium("999 years 400 days"), 0) - // Millennium must be taken from years and months checkEvaluation(Millennium("999 years 12 months"), 1) checkEvaluation(Millennium("1000 years -1 months"), 0) @@ -46,13 +43,10 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper test("century") { checkEvaluation(Century("0 years"), 0) checkEvaluation(Century("9999 years"), 99) - checkEvaluation(Century("1000 years"), 10) checkEvaluation(Century("-2000 years"), -20) - // Microseconds part must not be taken into account checkEvaluation(Century("99 years 400 days"), 0) - // Century must be taken from years and months checkEvaluation(Century("99 years 12 months"), 1) checkEvaluation(Century("100 years -1 months"), 0) @@ -61,13 +55,10 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper test("decade") { checkEvaluation(Decade("0 years"), 0) checkEvaluation(Decade("9999 years"), 999) - checkEvaluation(Decade("1000 years"), 100) checkEvaluation(Decade("-2000 years"), -200) - // Microseconds part must not be taken into account checkEvaluation(Decade("9 years 400 days"), 0) - // Decade must be taken from years and months checkEvaluation(Decade("9 years 12 months"), 1) checkEvaluation(Decade("10 years -1 months"), 0) @@ -76,13 +67,10 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper test("year") { checkEvaluation(Year("0 years"), 0) checkEvaluation(Year("9999 years"), 9999) - checkEvaluation(Year("1000 years"), 1000) checkEvaluation(Year("-2000 years"), -2000) - // Microseconds part must not be taken into account checkEvaluation(Year("9 years 400 days"), 9) - // Year must be taken from years and months checkEvaluation(Year("9 years 12 months"), 10) checkEvaluation(Year("10 years -1 months"), 9) From 8e4ca7d90fa283c892ef8b6c961c201216b2e7d0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:37:06 +0300 Subject: [PATCH 10/42] Add Month --- .../catalyst/expressions/intervalExpressions.scala | 14 +++++++++++++- .../spark/sql/catalyst/util/IntervalUtils.scala | 6 +++++- .../interval/IntervalExpressionsSuite.scala | 10 ++++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 19e3aedbd6fc..9f658f57fed6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -85,6 +85,18 @@ case class Quarter(child: Expression) extends UnaryExpression with ExpectsInputT } } +case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + override def dataType: DataType = IntegerType + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getMonth(date.asInstanceOf[CalendarInterval]) + } + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getMonth($c)") + } +} + object IntervalPart { def parseExtractField( @@ -97,7 +109,7 @@ object IntervalPart { case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) case "ISOYEAR" => errorHandleFunc case "QUARTER" | "QTR" => Quarter(source) -// case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) // case "WEEK" | "W" | "WEEKS" => WeekOfYear(source) // case "DAY" | "D" | "DAYS" => DayOfMonth(source) // case "DAYOFWEEK" => DayOfWeek(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 2fdec9fef615..eef5bc54de25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -42,7 +42,11 @@ object IntervalUtils { getYear(interval) / YEARS_PER_DECADE } + def getMonth(interval: CalendarInterval): Int = { + interval.months % MONTHS_PER_YEAR + } + def getQuarter(interval: CalendarInterval): Int = { - (interval.months % MONTHS_PER_YEAR) / MONTHS_PER_QUARTER + 1 + getMonth(interval) / MONTHS_PER_QUARTER + 1 } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 39f33a36a656..b86b607c8b9a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -88,4 +88,14 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Quarter("-2 years -3 months"), 0) checkEvaluation(Quarter("9999 years"), 1) } + + test("month") { + checkEvaluation(Month("0 year"), 0) + for (m <- -24 to 24) { + checkEvaluation(Month(s"$m months"), m % 12) + } + checkEvaluation(Month("1 year 10 months"), 10) + checkEvaluation(Month("-2 year -10 months"), -10) + checkEvaluation(Month("9999 years"), 0) + } } From 8ed01c7ad13e329b28bf89e83d8da7bdcfef503d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:38:09 +0300 Subject: [PATCH 11/42] Week is not supported by PostgreSQL --- .../spark/sql/catalyst/expressions/intervalExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 9f658f57fed6..5b5b0f3141fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -110,7 +110,7 @@ object IntervalPart { case "ISOYEAR" => errorHandleFunc case "QUARTER" | "QTR" => Quarter(source) case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) -// case "WEEK" | "W" | "WEEKS" => WeekOfYear(source) + case "WEEK" | "W" | "WEEKS" => errorHandleFunc // case "DAY" | "D" | "DAYS" => DayOfMonth(source) // case "DAYOFWEEK" => DayOfWeek(source) // case "DOW" => Subtract(DayOfWeek(source), Literal(1)) From f1eea12c76de2664c809ff1bf6fb59fd96d90b0c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:46:16 +0300 Subject: [PATCH 12/42] Remove not-supported fields --- .../sql/catalyst/expressions/intervalExpressions.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 5b5b0f3141fa..2a047b482cef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -107,15 +107,10 @@ object IntervalPart { case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) - case "ISOYEAR" => errorHandleFunc case "QUARTER" | "QTR" => Quarter(source) case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) case "WEEK" | "W" | "WEEKS" => errorHandleFunc // case "DAY" | "D" | "DAYS" => DayOfMonth(source) -// case "DAYOFWEEK" => DayOfWeek(source) -// case "DOW" => Subtract(DayOfWeek(source), Literal(1)) -// case "ISODOW" => Add(WeekDay(source), Literal(1)) -// case "DOY" => DayOfYear(source) // case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) // case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) // case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) @@ -124,6 +119,6 @@ object IntervalPart { // case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => // Microseconds(source) // case "EPOCH" => Epoch(source) -// case _ => errorHandleFunc + case _ => errorHandleFunc } } From 561f7894aa142e7f88e5973598a934f644488182 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 10:50:43 +0300 Subject: [PATCH 13/42] Week of interval is not supported --- .../spark/sql/catalyst/expressions/intervalExpressions.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 2a047b482cef..ca2b27c95bdf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -109,7 +109,6 @@ object IntervalPart { case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) case "QUARTER" | "QTR" => Quarter(source) case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) - case "WEEK" | "W" | "WEEKS" => errorHandleFunc // case "DAY" | "D" | "DAYS" => DayOfMonth(source) // case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) // case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) From 0a671a96a398e5d188360de90eff8f4cd80459ce Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 11:04:13 +0300 Subject: [PATCH 14/42] Add Day --- .../expressions/intervalExpressions.scala | 16 ++++++++++++++-- .../spark/sql/catalyst/util/IntervalUtils.scala | 4 ++++ .../interval/IntervalExpressionsSuite.scala | 12 ++++++++++++ 3 files changed, 30 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index ca2b27c95bdf..d41bb0243e99 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils -import org.apache.spark.sql.types.{AbstractDataType, CalendarIntervalType, DataType, IntegerType} +import org.apache.spark.sql.types.{AbstractDataType, CalendarIntervalType, DataType, IntegerType, LongType} import org.apache.spark.unsafe.types.CalendarInterval case class Millennium(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -97,6 +97,18 @@ case class Month(child: Expression) extends UnaryExpression with ExpectsInputTyp } } +case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + override def dataType: DataType = LongType + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getDay(date.asInstanceOf[CalendarInterval]) + } + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getDay($c)") + } +} + object IntervalPart { def parseExtractField( @@ -109,7 +121,7 @@ object IntervalPart { case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) case "QUARTER" | "QTR" => Quarter(source) case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) -// case "DAY" | "D" | "DAYS" => DayOfMonth(source) + case "DAY" | "D" | "DAYS" => Day(source) // case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) // case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) // case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index eef5bc54de25..7b60793c40b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -49,4 +49,8 @@ object IntervalUtils { def getQuarter(interval: CalendarInterval): Int = { getMonth(interval) / MONTHS_PER_QUARTER + 1 } + + def getDay(interval: CalendarInterval): Long = { + interval.microseconds / DateTimeUtils.MICROS_PER_DAY + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index b86b607c8b9a..e8d14632541c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -98,4 +98,16 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Month("-2 year -10 months"), -10) checkEvaluation(Month("9999 years"), 0) } + + test("day") { + checkEvaluation(Day("0 days"), 0L) + checkEvaluation(Day("1 days 100 seconds"), 1L) + checkEvaluation(Day("-1 days -100 seconds"), -1L) + checkEvaluation(Day("-365 days"), -365L) + checkEvaluation(Day("365 days"), 365L) + // Years and months must not be taken into account + checkEvaluation(Day("100 year 10 months 5 days"), 5L) + checkEvaluation(Day( + "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 31L) + } } From 894a6c7b0d5934a3a87eca89039af204ae380551 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 11:24:58 +0300 Subject: [PATCH 15/42] Add Hour --- .../expressions/intervalExpressions.scala | 16 ++++++++++++++-- .../spark/sql/catalyst/util/IntervalUtils.scala | 5 +++++ .../interval/IntervalExpressionsSuite.scala | 12 ++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index d41bb0243e99..16f616bca189 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -22,7 +22,7 @@ import java.util.Locale import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils -import org.apache.spark.sql.types.{AbstractDataType, CalendarIntervalType, DataType, IntegerType, LongType} +import org.apache.spark.sql.types.{AbstractDataType, ByteType, CalendarIntervalType, DataType, IntegerType, LongType} import org.apache.spark.unsafe.types.CalendarInterval case class Millennium(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -109,6 +109,18 @@ case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes } } +case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + override def dataType: DataType = ByteType + override protected def nullSafeEval(date: Any): Any = { + IntervalUtils.getHour(date.asInstanceOf[CalendarInterval]) + } + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val iu = IntervalUtils.getClass.getName.stripSuffix("$") + defineCodeGen(ctx, ev, c => s"$iu.getHour($c)") + } +} + object IntervalPart { def parseExtractField( @@ -122,7 +134,7 @@ object IntervalPart { case "QUARTER" | "QTR" => Quarter(source) case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) case "DAY" | "D" | "DAYS" => Day(source) -// case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) // case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) // case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) // case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7b60793c40b1..e67b59ca83d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -25,6 +25,7 @@ object IntervalUtils { val YEARS_PER_MILLENNIUM: Int = 1000 val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 + val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS def getYear(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -53,4 +54,8 @@ object IntervalUtils { def getDay(interval: CalendarInterval): Long = { interval.microseconds / DateTimeUtils.MICROS_PER_DAY } + + def getHour(interval: CalendarInterval): Byte = { + ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index e8d14632541c..f690e930abb1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -110,4 +110,16 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Day( "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 31L) } + + test("hour") { + checkEvaluation(Hour("0 hours"), 0.toByte) + checkEvaluation(Hour("1 hour"), 1.toByte) + checkEvaluation(Hour("-1 hour"), -1.toByte) + checkEvaluation(Hour("23 hours"), 23.toByte) + checkEvaluation(Hour("-23 hours"), -23.toByte) + // Years and months must not be taken into account + checkEvaluation(Hour("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(Hour( + "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 11.toByte) + } } From f86f4f542586614bd5a49735e1a783f062e2ccf1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 11:30:38 +0300 Subject: [PATCH 16/42] Change Month type to ByteType --- .../sql/catalyst/expressions/intervalExpressions.scala | 2 +- .../apache/spark/sql/catalyst/util/IntervalUtils.scala | 4 ++-- .../interval/IntervalExpressionsSuite.scala | 10 +++++----- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 16f616bca189..c92edb3b66bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -87,7 +87,7 @@ case class Quarter(child: Expression) extends UnaryExpression with ExpectsInputT case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType + override def dataType: DataType = ByteType override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getMonth(date.asInstanceOf[CalendarInterval]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index e67b59ca83d3..7908c358d75c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -43,8 +43,8 @@ object IntervalUtils { getYear(interval) / YEARS_PER_DECADE } - def getMonth(interval: CalendarInterval): Int = { - interval.months % MONTHS_PER_YEAR + def getMonth(interval: CalendarInterval): Byte = { + (interval.months % MONTHS_PER_YEAR).toByte } def getQuarter(interval: CalendarInterval): Int = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index f690e930abb1..5965402e7f96 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -90,13 +90,13 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper } test("month") { - checkEvaluation(Month("0 year"), 0) + checkEvaluation(Month("0 year"), 0.toByte) for (m <- -24 to 24) { - checkEvaluation(Month(s"$m months"), m % 12) + checkEvaluation(Month(s"$m months"), (m % 12).toByte) } - checkEvaluation(Month("1 year 10 months"), 10) - checkEvaluation(Month("-2 year -10 months"), -10) - checkEvaluation(Month("9999 years"), 0) + checkEvaluation(Month("1 year 10 months"), 10.toByte) + checkEvaluation(Month("-2 year -10 months"), -10.toByte) + checkEvaluation(Month("9999 years"), 0.toByte) } test("day") { From 283fd9929a01b43ca6668cfe53224102ff2b8d17 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 11:36:42 +0300 Subject: [PATCH 17/42] Change Quarter type to ByteType --- .../expressions/intervalExpressions.scala | 2 +- .../sql/catalyst/util/IntervalUtils.scala | 6 +++--- .../interval/IntervalExpressionsSuite.scala | 20 +++++++++---------- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index c92edb3b66bb..19c6e25d3a1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -75,7 +75,7 @@ case class Year(child: Expression) extends UnaryExpression with ExpectsInputType case class Quarter(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType + override def dataType: DataType = ByteType override protected def nullSafeEval(date: Any): Any = { IntervalUtils.getQuarter(date.asInstanceOf[CalendarInterval]) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7908c358d75c..9b4498b2981f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -21,7 +21,7 @@ import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { val MONTHS_PER_YEAR: Int = 12 - val MONTHS_PER_QUARTER: Int = 3 + val MONTHS_PER_QUARTER: Byte = 3 val YEARS_PER_MILLENNIUM: Int = 1000 val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 @@ -47,8 +47,8 @@ object IntervalUtils { (interval.months % MONTHS_PER_YEAR).toByte } - def getQuarter(interval: CalendarInterval): Int = { - getMonth(interval) / MONTHS_PER_QUARTER + 1 + def getQuarter(interval: CalendarInterval): Byte = { + (getMonth(interval) / MONTHS_PER_QUARTER + 1).toByte } def getDay(interval: CalendarInterval): Long = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 5965402e7f96..d09a50707427 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -77,16 +77,16 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper } test("quarter") { - checkEvaluation(Quarter("0 months"), 1) - checkEvaluation(Quarter("1 months"), 1) - checkEvaluation(Quarter("-1 months"), 1) - checkEvaluation(Quarter("2 months"), 1) - checkEvaluation(Quarter("-2 months"), 1) - checkEvaluation(Quarter("1 years -1 months"), 4) - checkEvaluation(Quarter("-1 years 1 months"), -2) - checkEvaluation(Quarter("2 years 3 months"), 2) - checkEvaluation(Quarter("-2 years -3 months"), 0) - checkEvaluation(Quarter("9999 years"), 1) + checkEvaluation(Quarter("0 months"), 1.toByte) + checkEvaluation(Quarter("1 months"), 1.toByte) + checkEvaluation(Quarter("-1 months"), 1.toByte) + checkEvaluation(Quarter("2 months"), 1.toByte) + checkEvaluation(Quarter("-2 months"), 1.toByte) + checkEvaluation(Quarter("1 years -1 months"), 4.toByte) + checkEvaluation(Quarter("-1 years 1 months"), -2.toByte) + checkEvaluation(Quarter("2 years 3 months"), 2.toByte) + checkEvaluation(Quarter("-2 years -3 months"), 0.toByte) + checkEvaluation(Quarter("9999 years"), 1.toByte) } test("month") { From 5e189ca28cc51559bbdd7b70a06b9c34d66ae276 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 12:29:34 +0300 Subject: [PATCH 18/42] Put common code to IntervalPart --- .../expressions/intervalExpressions.scala | 131 +++++++----------- 1 file changed, 49 insertions(+), 82 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 19c6e25d3a1c..bbe2744315e7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -25,101 +25,68 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.types.{AbstractDataType, ByteType, CalendarIntervalType, DataType, IntegerType, LongType} import org.apache.spark.unsafe.types.CalendarInterval -case class Millennium(child: Expression) extends UnaryExpression with ExpectsInputTypes { +abstract class IntervalPart( + child: Expression, + val dataType: DataType, + func: CalendarInterval => Any, + funcName: String) extends UnaryExpression with ExpectsInputTypes with Serializable { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getMillennium(date.asInstanceOf[CalendarInterval]) + override protected def nullSafeEval(interval: Any): Any = { + func(interval.asInstanceOf[CalendarInterval]) } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getMillennium($c)") + defineCodeGen(ctx, ev, c => s"$iu.$funcName($c)") } } -case class Century(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getCentury(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getCentury($c)") - } -} +case class Millennium(child: Expression) extends IntervalPart( + child, + IntegerType, + IntervalUtils.getMillennium, + "getMillennium") -case class Decade(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getDecade(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getDecade($c)") - } -} +case class Century(child: Expression) extends IntervalPart( + child, + IntegerType, + IntervalUtils.getCentury, + "getCentury") -case class Year(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = IntegerType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getYear(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getYear($c)") - } -} +case class Decade(child: Expression) extends IntervalPart( + child, + IntegerType, + IntervalUtils.getDecade, + "getDecade") -case class Quarter(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = ByteType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getQuarter(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getQuarter($c)") - } -} +case class Year(child: Expression) extends IntervalPart( + child, + IntegerType, + IntervalUtils.getYear, + "getYear") -case class Month(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = ByteType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getMonth(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getMonth($c)") - } -} +case class Quarter(child: Expression) extends IntervalPart( + child, + ByteType, + IntervalUtils.getQuarter, + "getQuarter") -case class Day(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = LongType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getDay(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getDay($c)") - } -} +case class Month(child: Expression) extends IntervalPart( + child, + ByteType, + IntervalUtils.getMonth, + "getMonth") -case class Hour(child: Expression) extends UnaryExpression with ExpectsInputTypes { - override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) - override def dataType: DataType = ByteType - override protected def nullSafeEval(date: Any): Any = { - IntervalUtils.getHour(date.asInstanceOf[CalendarInterval]) - } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val iu = IntervalUtils.getClass.getName.stripSuffix("$") - defineCodeGen(ctx, ev, c => s"$iu.getHour($c)") - } -} +case class Day(child: Expression) extends IntervalPart( + child, + LongType, + IntervalUtils.getDay, + "getDay") + +case class Hour(child: Expression) extends IntervalPart( + child, + ByteType, + IntervalUtils.getHour, + "getHour") object IntervalPart { From 62c21b9e7219953e5219d0b700e329a3477c397c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 12:38:07 +0300 Subject: [PATCH 19/42] Run scalafmt --- .../expressions/intervalExpressions.scala | 60 ++++++------------- 1 file changed, 18 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index bbe2744315e7..2b89771b8c25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -22,14 +22,18 @@ import java.util.Locale import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils -import org.apache.spark.sql.types.{AbstractDataType, ByteType, CalendarIntervalType, DataType, IntegerType, LongType} +import org.apache.spark.sql.catalyst.util.IntervalUtils._ +import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval abstract class IntervalPart( child: Expression, val dataType: DataType, func: CalendarInterval => Any, - funcName: String) extends UnaryExpression with ExpectsInputTypes with Serializable { + funcName: String) + extends UnaryExpression + with ExpectsInputTypes + with Serializable { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) override protected def nullSafeEval(interval: Any): Any = { func(interval.asInstanceOf[CalendarInterval]) @@ -40,53 +44,25 @@ abstract class IntervalPart( } } -case class Millennium(child: Expression) extends IntervalPart( - child, - IntegerType, - IntervalUtils.getMillennium, - "getMillennium") +case class Millennium(child: Expression) + extends IntervalPart(child, IntegerType, getMillennium, "getMillennium") -case class Century(child: Expression) extends IntervalPart( - child, - IntegerType, - IntervalUtils.getCentury, - "getCentury") +case class Century(child: Expression) + extends IntervalPart(child, IntegerType, getCentury, "getCentury") -case class Decade(child: Expression) extends IntervalPart( - child, - IntegerType, - IntervalUtils.getDecade, - "getDecade") +case class Decade(child: Expression) + extends IntervalPart(child, IntegerType, getDecade, "getDecade") -case class Year(child: Expression) extends IntervalPart( - child, - IntegerType, - IntervalUtils.getYear, - "getYear") +case class Year(child: Expression) extends IntervalPart(child, IntegerType, getYear, "getYear") -case class Quarter(child: Expression) extends IntervalPart( - child, - ByteType, - IntervalUtils.getQuarter, - "getQuarter") +case class Quarter(child: Expression) + extends IntervalPart(child, ByteType, getQuarter, "getQuarter") -case class Month(child: Expression) extends IntervalPart( - child, - ByteType, - IntervalUtils.getMonth, - "getMonth") +case class Month(child: Expression) extends IntervalPart(child, ByteType, getMonth, "getMonth") -case class Day(child: Expression) extends IntervalPart( - child, - LongType, - IntervalUtils.getDay, - "getDay") +case class Day(child: Expression) extends IntervalPart(child, LongType, getDay, "getDay") -case class Hour(child: Expression) extends IntervalPart( - child, - ByteType, - IntervalUtils.getHour, - "getHour") +case class Hour(child: Expression) extends IntervalPart(child, ByteType, getHour, "getHour") object IntervalPart { From a4fbb5e314141bfc5a60d64ab6164c9585d70946 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 13:20:22 +0300 Subject: [PATCH 20/42] Add Minute --- .../catalyst/expressions/intervalExpressions.scala | 4 +++- .../spark/sql/catalyst/util/IntervalUtils.scala | 5 +++++ .../interval/IntervalExpressionsSuite.scala | 12 ++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 2b89771b8c25..3ffde902b023 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -64,6 +64,8 @@ case class Day(child: Expression) extends IntervalPart(child, LongType, getDay, case class Hour(child: Expression) extends IntervalPart(child, ByteType, getHour, "getHour") +case class Minute(child: Expression) extends IntervalPart(child, ByteType, getMinute, "getMinute") + object IntervalPart { def parseExtractField( @@ -78,7 +80,7 @@ object IntervalPart { case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) case "DAY" | "D" | "DAYS" => Day(source) case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) -// case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) // case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) // case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => // Milliseconds(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 9b4498b2981f..3d8ad5a19031 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -26,6 +26,7 @@ object IntervalUtils { val YEARS_PER_CENTURY: Int = 100 val YEARS_PER_DECADE: Int = 10 val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS def getYear(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -58,4 +59,8 @@ object IntervalUtils { def getHour(interval: CalendarInterval): Byte = { ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte } + + def getMinute(interval: CalendarInterval): Byte = { + ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index d09a50707427..1b215174eb68 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -122,4 +122,16 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Hour( "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 11.toByte) } + + test("minute") { + checkEvaluation(Minute("0 minute"), 0.toByte) + checkEvaluation(Minute("1 minute"), 1.toByte) + checkEvaluation(Minute("-1 minute"), -1.toByte) + checkEvaluation(Minute("59 minute"), 59.toByte) + checkEvaluation(Minute("-59 minute"), -59.toByte) + // Years and months must not be taken into account + checkEvaluation(Minute("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(Minute( + "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 59.toByte) + } } From e1c94159895bbcfa9690ecf21f3f95abba00f25a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 14:43:24 +0300 Subject: [PATCH 21/42] Add Second --- .../expressions/intervalExpressions.scala | 5 +++- .../sql/catalyst/util/IntervalUtils.scala | 5 ++++ .../interval/IntervalExpressionsSuite.scala | 26 ++++++++++++++----- 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 3ffde902b023..30be34953b23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -66,6 +66,9 @@ case class Hour(child: Expression) extends IntervalPart(child, ByteType, getHour case class Minute(child: Expression) extends IntervalPart(child, ByteType, getMinute, "getMinute") +case class Second(child: Expression) extends IntervalPart( + child, DecimalType(8, 6), getSecond, "getSecond") + object IntervalPart { def parseExtractField( @@ -81,7 +84,7 @@ object IntervalPart { case "DAY" | "D" | "DAYS" => Day(source) case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) -// case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) // case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => // Milliseconds(source) // case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 3d8ad5a19031..a3912861200d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.util +import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { @@ -63,4 +64,8 @@ object IntervalUtils { def getMinute(interval: CalendarInterval): Byte = { ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte } + + def getSecond(interval: CalendarInterval): Decimal = { + Decimal(interval.microseconds % MICROS_PER_MINUTE, 8, 6) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 1b215174eb68..20a9f23fbcba 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.expressions.interval import scala.language.implicitConversions - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -99,6 +99,9 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Month("9999 years"), 0.toByte) } + private val largeInterval: String = + "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds" + test("day") { checkEvaluation(Day("0 days"), 0L) checkEvaluation(Day("1 days 100 seconds"), 1L) @@ -107,8 +110,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Day("365 days"), 365L) // Years and months must not be taken into account checkEvaluation(Day("100 year 10 months 5 days"), 5L) - checkEvaluation(Day( - "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 31L) + checkEvaluation(Day(largeInterval), 31L) } test("hour") { @@ -119,8 +121,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Hour("-23 hours"), -23.toByte) // Years and months must not be taken into account checkEvaluation(Hour("100 year 10 months 10 hours"), 10.toByte) - checkEvaluation(Hour( - "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 11.toByte) + checkEvaluation(Hour(largeInterval), 11.toByte) } test("minute") { @@ -131,7 +132,18 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Minute("-59 minute"), -59.toByte) // Years and months must not be taken into account checkEvaluation(Minute("100 year 10 months 10 minutes"), 10.toByte) - checkEvaluation(Minute( - "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds"), 59.toByte) + checkEvaluation(Minute(largeInterval), 59.toByte) + } + + test("second") { + checkEvaluation(Second("0 second"), Decimal(0, 8, 6)) + checkEvaluation(Second("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(Second("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(Second("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(Second("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + // Years and months must not be taken into account + checkEvaluation(Second("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(Second(largeInterval), Decimal(59.0, 8, 6)) + checkEvaluation(Second("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) } } From 7f4100f6df66ccad6e2b5834d957265b72026171 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 14:44:12 +0300 Subject: [PATCH 22/42] Run scalafmt --- .../spark/sql/catalyst/expressions/intervalExpressions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 30be34953b23..f86de5e08b0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -66,8 +66,8 @@ case class Hour(child: Expression) extends IntervalPart(child, ByteType, getHour case class Minute(child: Expression) extends IntervalPart(child, ByteType, getMinute, "getMinute") -case class Second(child: Expression) extends IntervalPart( - child, DecimalType(8, 6), getSecond, "getSecond") +case class Second(child: Expression) + extends IntervalPart(child, DecimalType(8, 6), getSecond, "getSecond") object IntervalPart { From f3cf7f01e060f75611768ef80c1ebce36561aded Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 16:17:43 +0300 Subject: [PATCH 23/42] Refactoring --- .../expressions/intervalExpressions.scala | 51 +++--- .../sql/catalyst/util/IntervalUtils.scala | 28 +-- .../interval/IntervalExpressionsSuite.scala | 167 +++++++++--------- 3 files changed, 124 insertions(+), 122 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index f86de5e08b0a..87277d62e237 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -44,30 +44,31 @@ abstract class IntervalPart( } } -case class Millennium(child: Expression) - extends IntervalPart(child, IntegerType, getMillennium, "getMillennium") +case class Millenniums(child: Expression) + extends IntervalPart(child, IntegerType, getMillenniums, "getMillenniums") -case class Century(child: Expression) - extends IntervalPart(child, IntegerType, getCentury, "getCentury") +case class Centuries(child: Expression) + extends IntervalPart(child, IntegerType, getCenturies, "getCenturies") -case class Decade(child: Expression) - extends IntervalPart(child, IntegerType, getDecade, "getDecade") +case class Decades(child: Expression) + extends IntervalPart(child, IntegerType, getDecades, "getDecades") -case class Year(child: Expression) extends IntervalPart(child, IntegerType, getYear, "getYear") +case class Years(child: Expression) extends IntervalPart(child, IntegerType, getYears, "getYears") -case class Quarter(child: Expression) - extends IntervalPart(child, ByteType, getQuarter, "getQuarter") +case class Quarters(child: Expression) + extends IntervalPart(child, ByteType, getQuarters, "getQuarters") -case class Month(child: Expression) extends IntervalPart(child, ByteType, getMonth, "getMonth") +case class Months(child: Expression) extends IntervalPart(child, ByteType, getMonths, "getMonths") -case class Day(child: Expression) extends IntervalPart(child, LongType, getDay, "getDay") +case class Days(child: Expression) extends IntervalPart(child, LongType, getDays, "getDays") -case class Hour(child: Expression) extends IntervalPart(child, ByteType, getHour, "getHour") +case class Hours(child: Expression) extends IntervalPart(child, ByteType, getHours, "getHours") -case class Minute(child: Expression) extends IntervalPart(child, ByteType, getMinute, "getMinute") +case class Minutes(child: Expression) + extends IntervalPart(child, ByteType, getMinutes, "getMinutes") -case class Second(child: Expression) - extends IntervalPart(child, DecimalType(8, 6), getSecond, "getSecond") +case class Seconds(child: Expression) + extends IntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") object IntervalPart { @@ -75,16 +76,16 @@ object IntervalPart { extractField: String, source: Expression, errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { - case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millennium(source) - case "CENTURY" | "CENTURIES" | "C" | "CENT" => Century(source) - case "DECADE" | "DECADES" | "DEC" | "DECS" => Decade(source) - case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Year(source) - case "QUARTER" | "QTR" => Quarter(source) - case "MONTH" | "MON" | "MONS" | "MONTHS" => Month(source) - case "DAY" | "D" | "DAYS" => Day(source) - case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hour(source) - case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minute(source) - case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Second(source) + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millenniums(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => Centuries(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => Decades(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Years(source) + case "QUARTER" | "QTR" => Quarters(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => Months(source) + case "DAY" | "D" | "DAYS" => Days(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hours(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minutes(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Seconds(source) // case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => // Milliseconds(source) // case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index a3912861200d..7f2e39fc7c4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -29,43 +29,43 @@ object IntervalUtils { val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS - def getYear(interval: CalendarInterval): Int = { + def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR } - def getMillennium(interval: CalendarInterval): Int = { - getYear(interval) / YEARS_PER_MILLENNIUM + def getMillenniums(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_MILLENNIUM } - def getCentury(interval: CalendarInterval): Int = { - getYear(interval) / YEARS_PER_CENTURY + def getCenturies(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_CENTURY } - def getDecade(interval: CalendarInterval): Int = { - getYear(interval) / YEARS_PER_DECADE + def getDecades(interval: CalendarInterval): Int = { + getYears(interval) / YEARS_PER_DECADE } - def getMonth(interval: CalendarInterval): Byte = { + def getMonths(interval: CalendarInterval): Byte = { (interval.months % MONTHS_PER_YEAR).toByte } - def getQuarter(interval: CalendarInterval): Byte = { - (getMonth(interval) / MONTHS_PER_QUARTER + 1).toByte + def getQuarters(interval: CalendarInterval): Byte = { + (getMonths(interval) / MONTHS_PER_QUARTER + 1).toByte } - def getDay(interval: CalendarInterval): Long = { + def getDays(interval: CalendarInterval): Long = { interval.microseconds / DateTimeUtils.MICROS_PER_DAY } - def getHour(interval: CalendarInterval): Byte = { + def getHours(interval: CalendarInterval): Byte = { ((interval.microseconds % DateTimeUtils.MICROS_PER_DAY) / MICROS_PER_HOUR).toByte } - def getMinute(interval: CalendarInterval): Byte = { + def getMinutes(interval: CalendarInterval): Byte = { ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte } - def getSecond(interval: CalendarInterval): Decimal = { + def getSeconds(interval: CalendarInterval): Decimal = { Decimal(interval.microseconds % MICROS_PER_MINUTE, 8, 6) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 20a9f23fbcba..68c752b5c443 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -18,132 +18,133 @@ package org.apache.spark.sql.catalyst.expressions.interval import scala.language.implicitConversions + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval -class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { +class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { implicit def interval(s: String): Literal = { Literal(CalendarInterval.fromString("interval " + s)) } - test("millennium") { - checkEvaluation(Millennium("0 years"), 0) - checkEvaluation(Millennium("9999 years"), 9) - checkEvaluation(Millennium("1000 years"), 1) - checkEvaluation(Millennium("-2000 years"), -2) + test("millenniums") { + checkEvaluation(Millenniums("0 years"), 0) + checkEvaluation(Millenniums("9999 years"), 9) + checkEvaluation(Millenniums("1000 years"), 1) + checkEvaluation(Millenniums("-2000 years"), -2) // Microseconds part must not be taken into account - checkEvaluation(Millennium("999 years 400 days"), 0) + checkEvaluation(Millenniums("999 years 400 days"), 0) // Millennium must be taken from years and months - checkEvaluation(Millennium("999 years 12 months"), 1) - checkEvaluation(Millennium("1000 years -1 months"), 0) + checkEvaluation(Millenniums("999 years 12 months"), 1) + checkEvaluation(Millenniums("1000 years -1 months"), 0) } - test("century") { - checkEvaluation(Century("0 years"), 0) - checkEvaluation(Century("9999 years"), 99) - checkEvaluation(Century("1000 years"), 10) - checkEvaluation(Century("-2000 years"), -20) + test("centuries") { + checkEvaluation(Centuries("0 years"), 0) + checkEvaluation(Centuries("9999 years"), 99) + checkEvaluation(Centuries("1000 years"), 10) + checkEvaluation(Centuries("-2000 years"), -20) // Microseconds part must not be taken into account - checkEvaluation(Century("99 years 400 days"), 0) + checkEvaluation(Centuries("99 years 400 days"), 0) // Century must be taken from years and months - checkEvaluation(Century("99 years 12 months"), 1) - checkEvaluation(Century("100 years -1 months"), 0) + checkEvaluation(Centuries("99 years 12 months"), 1) + checkEvaluation(Centuries("100 years -1 months"), 0) } - test("decade") { - checkEvaluation(Decade("0 years"), 0) - checkEvaluation(Decade("9999 years"), 999) - checkEvaluation(Decade("1000 years"), 100) - checkEvaluation(Decade("-2000 years"), -200) + test("decades") { + checkEvaluation(Decades("0 years"), 0) + checkEvaluation(Decades("9999 years"), 999) + checkEvaluation(Decades("1000 years"), 100) + checkEvaluation(Decades("-2000 years"), -200) // Microseconds part must not be taken into account - checkEvaluation(Decade("9 years 400 days"), 0) + checkEvaluation(Decades("9 years 400 days"), 0) // Decade must be taken from years and months - checkEvaluation(Decade("9 years 12 months"), 1) - checkEvaluation(Decade("10 years -1 months"), 0) + checkEvaluation(Decades("9 years 12 months"), 1) + checkEvaluation(Decades("10 years -1 months"), 0) } - test("year") { - checkEvaluation(Year("0 years"), 0) - checkEvaluation(Year("9999 years"), 9999) - checkEvaluation(Year("1000 years"), 1000) - checkEvaluation(Year("-2000 years"), -2000) + test("years") { + checkEvaluation(Years("0 years"), 0) + checkEvaluation(Years("9999 years"), 9999) + checkEvaluation(Years("1000 years"), 1000) + checkEvaluation(Years("-2000 years"), -2000) // Microseconds part must not be taken into account - checkEvaluation(Year("9 years 400 days"), 9) + checkEvaluation(Years("9 years 400 days"), 9) // Year must be taken from years and months - checkEvaluation(Year("9 years 12 months"), 10) - checkEvaluation(Year("10 years -1 months"), 9) + checkEvaluation(Years("9 years 12 months"), 10) + checkEvaluation(Years("10 years -1 months"), 9) } - test("quarter") { - checkEvaluation(Quarter("0 months"), 1.toByte) - checkEvaluation(Quarter("1 months"), 1.toByte) - checkEvaluation(Quarter("-1 months"), 1.toByte) - checkEvaluation(Quarter("2 months"), 1.toByte) - checkEvaluation(Quarter("-2 months"), 1.toByte) - checkEvaluation(Quarter("1 years -1 months"), 4.toByte) - checkEvaluation(Quarter("-1 years 1 months"), -2.toByte) - checkEvaluation(Quarter("2 years 3 months"), 2.toByte) - checkEvaluation(Quarter("-2 years -3 months"), 0.toByte) - checkEvaluation(Quarter("9999 years"), 1.toByte) + test("quarters") { + checkEvaluation(Quarters("0 months"), 1.toByte) + checkEvaluation(Quarters("1 months"), 1.toByte) + checkEvaluation(Quarters("-1 months"), 1.toByte) + checkEvaluation(Quarters("2 months"), 1.toByte) + checkEvaluation(Quarters("-2 months"), 1.toByte) + checkEvaluation(Quarters("1 years -1 months"), 4.toByte) + checkEvaluation(Quarters("-1 years 1 months"), -2.toByte) + checkEvaluation(Quarters("2 years 3 months"), 2.toByte) + checkEvaluation(Quarters("-2 years -3 months"), 0.toByte) + checkEvaluation(Quarters("9999 years"), 1.toByte) } - test("month") { - checkEvaluation(Month("0 year"), 0.toByte) + test("months") { + checkEvaluation(Months("0 year"), 0.toByte) for (m <- -24 to 24) { - checkEvaluation(Month(s"$m months"), (m % 12).toByte) + checkEvaluation(Months(s"$m months"), (m % 12).toByte) } - checkEvaluation(Month("1 year 10 months"), 10.toByte) - checkEvaluation(Month("-2 year -10 months"), -10.toByte) - checkEvaluation(Month("9999 years"), 0.toByte) + checkEvaluation(Months("1 year 10 months"), 10.toByte) + checkEvaluation(Months("-2 year -10 months"), -10.toByte) + checkEvaluation(Months("9999 years"), 0.toByte) } private val largeInterval: String = "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds" - test("day") { - checkEvaluation(Day("0 days"), 0L) - checkEvaluation(Day("1 days 100 seconds"), 1L) - checkEvaluation(Day("-1 days -100 seconds"), -1L) - checkEvaluation(Day("-365 days"), -365L) - checkEvaluation(Day("365 days"), 365L) + test("days") { + checkEvaluation(Days("0 days"), 0L) + checkEvaluation(Days("1 days 100 seconds"), 1L) + checkEvaluation(Days("-1 days -100 seconds"), -1L) + checkEvaluation(Days("-365 days"), -365L) + checkEvaluation(Days("365 days"), 365L) // Years and months must not be taken into account - checkEvaluation(Day("100 year 10 months 5 days"), 5L) - checkEvaluation(Day(largeInterval), 31L) + checkEvaluation(Days("100 year 10 months 5 days"), 5L) + checkEvaluation(Days(largeInterval), 31L) } - test("hour") { - checkEvaluation(Hour("0 hours"), 0.toByte) - checkEvaluation(Hour("1 hour"), 1.toByte) - checkEvaluation(Hour("-1 hour"), -1.toByte) - checkEvaluation(Hour("23 hours"), 23.toByte) - checkEvaluation(Hour("-23 hours"), -23.toByte) + test("hours") { + checkEvaluation(Hours("0 hours"), 0.toByte) + checkEvaluation(Hours("1 hour"), 1.toByte) + checkEvaluation(Hours("-1 hour"), -1.toByte) + checkEvaluation(Hours("23 hours"), 23.toByte) + checkEvaluation(Hours("-23 hours"), -23.toByte) // Years and months must not be taken into account - checkEvaluation(Hour("100 year 10 months 10 hours"), 10.toByte) - checkEvaluation(Hour(largeInterval), 11.toByte) + checkEvaluation(Hours("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(Hours(largeInterval), 11.toByte) } - test("minute") { - checkEvaluation(Minute("0 minute"), 0.toByte) - checkEvaluation(Minute("1 minute"), 1.toByte) - checkEvaluation(Minute("-1 minute"), -1.toByte) - checkEvaluation(Minute("59 minute"), 59.toByte) - checkEvaluation(Minute("-59 minute"), -59.toByte) + test("minutes") { + checkEvaluation(Minutes("0 minute"), 0.toByte) + checkEvaluation(Minutes("1 minute"), 1.toByte) + checkEvaluation(Minutes("-1 minute"), -1.toByte) + checkEvaluation(Minutes("59 minute"), 59.toByte) + checkEvaluation(Minutes("-59 minute"), -59.toByte) // Years and months must not be taken into account - checkEvaluation(Minute("100 year 10 months 10 minutes"), 10.toByte) - checkEvaluation(Minute(largeInterval), 59.toByte) + checkEvaluation(Minutes("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(Minutes(largeInterval), 59.toByte) } - test("second") { - checkEvaluation(Second("0 second"), Decimal(0, 8, 6)) - checkEvaluation(Second("1 second"), Decimal(1.0, 8, 6)) - checkEvaluation(Second("-1 second"), Decimal(-1.0, 8, 6)) - checkEvaluation(Second("1 minute 59 second"), Decimal(59.0, 8, 6)) - checkEvaluation(Second("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + test("seconds") { + checkEvaluation(Seconds("0 second"), Decimal(0, 8, 6)) + checkEvaluation(Seconds("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(Seconds("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(Seconds("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(Seconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) // Years and months must not be taken into account - checkEvaluation(Second("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) - checkEvaluation(Second(largeInterval), Decimal(59.0, 8, 6)) - checkEvaluation(Second("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) + checkEvaluation(Seconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(Seconds(largeInterval), Decimal(59.0, 8, 6)) + checkEvaluation(Seconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) } } From 9262f9d85d7cb130553ca168fdf03b83b8f36e3d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 16:23:33 +0300 Subject: [PATCH 24/42] Add a test for overflow --- .../catalyst/expressions/interval/IntervalExpressionsSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 68c752b5c443..11f99ece460f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -146,5 +146,6 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Seconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) checkEvaluation(Seconds(largeInterval), Decimal(59.0, 8, 6)) checkEvaluation(Seconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) + checkEvaluation(Seconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) } } From b9890ecd23ca319a02d08d00022e1604258e22f1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 16:38:38 +0300 Subject: [PATCH 25/42] Add Milliseconds --- .../expressions/intervalExpressions.scala | 7 +++++-- .../spark/sql/catalyst/util/IntervalUtils.scala | 10 +++++++++- .../interval/IntervalExpressionsSuite.scala | 16 ++++++++++++++-- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 87277d62e237..3d9e6ea28f4e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -70,6 +70,9 @@ case class Minutes(child: Expression) case class Seconds(child: Expression) extends IntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") +case class Milliseconds(child: Expression) + extends IntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") + object IntervalPart { def parseExtractField( @@ -86,8 +89,8 @@ object IntervalPart { case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hours(source) case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minutes(source) case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Seconds(source) -// case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => -// Milliseconds(source) + case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => + Milliseconds(source) // case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => // Microseconds(source) // case "EPOCH" => Epoch(source) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 7f2e39fc7c4d..b86e1e4e355e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -65,7 +65,15 @@ object IntervalUtils { ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte } + private def getMicrosInMinute(interval: CalendarInterval): Long = { + interval.microseconds % MICROS_PER_MINUTE + } + def getSeconds(interval: CalendarInterval): Decimal = { - Decimal(interval.microseconds % MICROS_PER_MINUTE, 8, 6) + Decimal(getMicrosInMinute(interval), 8, 6) + } + + def getMilliseconds(interval: CalendarInterval): Decimal = { + Decimal(getMicrosInMinute(interval), 8, 3) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 11f99ece460f..64a52fa320ef 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -101,7 +101,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } private val largeInterval: String = - "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds" + "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds 999 millisecond" test("days") { checkEvaluation(Days("0 days"), 0L) @@ -144,8 +144,20 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Seconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) // Years and months must not be taken into account checkEvaluation(Seconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) - checkEvaluation(Seconds(largeInterval), Decimal(59.0, 8, 6)) + checkEvaluation(Seconds(largeInterval), Decimal(59.999, 8, 6)) checkEvaluation(Seconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) checkEvaluation(Seconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) } + + test("milliseconds") { + checkEvaluation(Milliseconds("0 milliseconds"), Decimal(0, 8, 3)) + checkEvaluation(Milliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) + checkEvaluation(Milliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) + checkEvaluation(Milliseconds("1 second 999 milliseconds"), Decimal(1999.0, 8, 3)) + checkEvaluation(Milliseconds("999 milliseconds 1 microsecond"), Decimal(999.001, 8, 3)) + checkEvaluation(Milliseconds("-1 second -999 milliseconds"), Decimal(-1999.0, 8, 3)) + // Years and months must not be taken into account + checkEvaluation(Milliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) + checkEvaluation(Milliseconds(largeInterval), Decimal(59999.0, 8, 3)) + } } From bea3faf8587673a4ae45f57aa5820c68e71aaa1f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 16:52:09 +0300 Subject: [PATCH 26/42] Add Microseconds --- .../expressions/intervalExpressions.scala | 7 +++++-- .../sql/catalyst/util/IntervalUtils.scala | 6 +++--- .../interval/IntervalExpressionsSuite.scala | 20 +++++++++++++++---- 3 files changed, 24 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 3d9e6ea28f4e..5b988702bbfb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -73,6 +73,9 @@ case class Seconds(child: Expression) case class Milliseconds(child: Expression) extends IntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") +case class Microseconds(child: Expression) + extends IntervalPart(child, LongType, getMicroseconds, "getMicroseconds") + object IntervalPart { def parseExtractField( @@ -91,8 +94,8 @@ object IntervalPart { case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Seconds(source) case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => Milliseconds(source) -// case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => -// Microseconds(source) + case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => + Microseconds(source) // case "EPOCH" => Epoch(source) case _ => errorHandleFunc } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index b86e1e4e355e..f1d177b12134 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -65,15 +65,15 @@ object IntervalUtils { ((interval.microseconds % MICROS_PER_HOUR) / MICROS_PER_MINUTE).toByte } - private def getMicrosInMinute(interval: CalendarInterval): Long = { + def getMicroseconds(interval: CalendarInterval): Long = { interval.microseconds % MICROS_PER_MINUTE } def getSeconds(interval: CalendarInterval): Decimal = { - Decimal(getMicrosInMinute(interval), 8, 6) + Decimal(getMicroseconds(interval), 8, 6) } def getMilliseconds(interval: CalendarInterval): Decimal = { - Decimal(getMicrosInMinute(interval), 8, 3) + Decimal(getMicroseconds(interval), 8, 3) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 64a52fa320ef..cba8c9711a5d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -100,8 +100,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Months("9999 years"), 0.toByte) } - private val largeInterval: String = - "9999 years 11 months 31 days 11 hours 59 minutes 59 seconds 999 millisecond" + private val largeInterval: String = "9999 years 11 months " + + "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" test("days") { checkEvaluation(Days("0 days"), 0L) @@ -144,7 +144,7 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Seconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) // Years and months must not be taken into account checkEvaluation(Seconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) - checkEvaluation(Seconds(largeInterval), Decimal(59.999, 8, 6)) + checkEvaluation(Seconds(largeInterval), Decimal(59.999999, 8, 6)) checkEvaluation(Seconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) checkEvaluation(Seconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) } @@ -158,6 +158,18 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Milliseconds("-1 second -999 milliseconds"), Decimal(-1999.0, 8, 3)) // Years and months must not be taken into account checkEvaluation(Milliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) - checkEvaluation(Milliseconds(largeInterval), Decimal(59999.0, 8, 3)) + checkEvaluation(Milliseconds(largeInterval), Decimal(59999.999, 8, 3)) + } + + test("microseconds") { + checkEvaluation(Microseconds("0 microseconds"), 0L) + checkEvaluation(Microseconds("1 microseconds"), 1L) + checkEvaluation(Microseconds("-1 microseconds"), -1L) + checkEvaluation(Microseconds("1 second 999 microseconds"), 1000999L) + checkEvaluation(Microseconds("999 milliseconds 1 microseconds"), 999001L) + checkEvaluation(Microseconds("-1 second -999 microseconds"), -1000999L) + // Years and months must not be taken into account + checkEvaluation(Microseconds("11 year 1 microseconds"), 1L) + checkEvaluation(Microseconds(largeInterval), 59999999L) } } From 77e0fb36ce6ee6011fb2f99cab4673e3a434c160 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 17:33:54 +0300 Subject: [PATCH 27/42] Add Epoch --- .../catalyst/expressions/intervalExpressions.scala | 7 ++++++- .../spark/sql/catalyst/util/IntervalUtils.scala | 12 ++++++++++++ .../interval/IntervalExpressionsSuite.scala | 8 ++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 5b988702bbfb..406412eed8d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -76,6 +76,11 @@ case class Milliseconds(child: Expression) case class Microseconds(child: Expression) extends IntervalPart(child, LongType, getMicroseconds, "getMicroseconds") +// Number of seconds in 10000 years is 315576000001 (30 days per one month) +// which is 12 digits + 6 digits for the fractional part of seconds. +case class Epoch(child: Expression) + extends IntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") + object IntervalPart { def parseExtractField( @@ -96,7 +101,7 @@ object IntervalPart { Milliseconds(source) case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => Microseconds(source) -// case "EPOCH" => Epoch(source) + case "EPOCH" => Epoch(source) case _ => errorHandleFunc } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index f1d177b12134..78d188f81f62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -28,6 +28,10 @@ object IntervalUtils { val YEARS_PER_DECADE: Int = 10 val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + val DAYS_PER_MONTH: Byte = 30 + val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + /* 365.25 days per year assumes leap year every four years */ + val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -76,4 +80,12 @@ object IntervalUtils { def getMilliseconds(interval: CalendarInterval): Decimal = { Decimal(getMicroseconds(interval), 8, 3) } + + // Returns total number of seconds with microseconds fractional part in the given interval. + def getEpoch(interval: CalendarInterval): Decimal = { + var result = interval.microseconds + result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) + result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) + Decimal(result, 18, 6) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index cba8c9711a5d..58eebb0a4d6f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -172,4 +172,12 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Microseconds("11 year 1 microseconds"), 1L) checkEvaluation(Microseconds(largeInterval), 59999999L) } + + test("epoch") { + checkEvaluation(Epoch("0 months"), Decimal(0.0, 18, 6)) + checkEvaluation(Epoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(Epoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(Epoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation(Epoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) + } } From 58017a7833be48fb9c3aaceb22e0706357fb4672 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 18:11:43 +0300 Subject: [PATCH 28/42] Support intervals by date_part --- .../catalyst/expressions/datetimeExpressions.scala | 13 +++++++++---- .../interval/IntervalExpressionsSuite.scala | 6 +++++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 0fffffea1d82..74901625c0e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.time.{DateTimeException, Instant, LocalDate, LocalDateTime, ZoneId} +import java.time.{DateTimeException, LocalDate, LocalDateTime, ZoneId} import java.time.temporal.IsoFields import java.util.{Locale, TimeZone} @@ -30,6 +30,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.expressions.interval.IntervalPart import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf @@ -2056,9 +2057,13 @@ case class DatePart(field: Expression, source: Expression, child: Expression) Literal(null, DoubleType) } else { val fieldStr = fieldEval.asInstanceOf[UTF8String].toString - DatePart.parseExtractField(fieldStr, source, { - throw new AnalysisException(s"Literals of type '$fieldStr' are currently not supported.") - }) + val errMsg = s"Literals of type '$fieldStr' are currently not supported " + + s"for the ${source.dataType.catalogString} type." + if (source.dataType == CalendarIntervalType) { + IntervalPart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) + } else { + DatePart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) + } } }) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 58eebb0a4d6f..9de38611b1df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.interval import scala.language.implicitConversions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} +import org.apache.spark.sql.catalyst.expressions.{DatePart, ExpressionEvalHelper, Literal} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval @@ -180,4 +180,8 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Epoch("-1 year"), Decimal(-31557600.0, 18, 6)) checkEvaluation(Epoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) } + + test("date_part from interval") { + checkEvaluation(new DatePart(Literal("months"), Literal(new CalendarInterval(10, 0))), 10) + } } From f5620b315ae0e3863963527d583e75b5bb4715ad Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 18:47:15 +0300 Subject: [PATCH 29/42] Update comments for DatePart --- .../expressions/datetimeExpressions.scala | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 74901625c0e9..0d518104fb97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -2009,10 +2009,11 @@ object DatePart { } @ExpressionDescription( - usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp.", + usage = "_FUNC_(field, source) - Extracts a part of the date/timestamp or interval source.", arguments = """ Arguments: - * field - selects which part of the source should be extracted. Supported string values are: + * field - selects which part of the source should be extracted. + Supported string values of `field` for dates and timestamps are: ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), "CENTURY", ("CENTURIES", "C", "CENT"), "DECADE", ("DECADES", "DEC", "DECS"), @@ -2032,7 +2033,21 @@ object DatePart { "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), "EPOCH"] - * source - a date (or timestamp) column from where `field` should be extracted + Supported string values of `field` for intervals are: + ["MILLENNIUM", ("MILLENNIA", "MIL", "MILS"), + "CENTURY", ("CENTURIES", "C", "CENT"), + "DECADE", ("DECADES", "DEC", "DECS"), + "YEAR", ("Y", "YEARS", "YR", "YRS"), + "QUARTER", ("QTR"), + "MONTH", ("MON", "MONS", "MONTHS"), + "DAY", ("D", "DAYS"), + "HOUR", ("H", "HOURS", "HR", "HRS"), + "MINUTE", ("M", "MIN", "MINS", "MINUTES"), + "SECOND", ("S", "SEC", "SECONDS", "SECS"), + "MILLISECONDS", ("MSEC", "MSECS", "MILLISECON", "MSECONDS", "MS"), + "MICROSECONDS", ("USEC", "USECS", "USECONDS", "MICROSECON", "US"), + "EPOCH"] + * source - a date/timestamp or interval column from where `field` should be extracted """, examples = """ Examples: @@ -2042,6 +2057,10 @@ object DatePart { 33 > SELECT _FUNC_('doy', DATE'2019-08-12'); 224 + > SELECT _FUNC_('days', interval 1 year 10 months 5 days); + 5 + > SELECT _FUNC_('seconds', interval 30 seconds 1 milliseconds 1 microseconds); + 30.001001 """, since = "3.0.0") case class DatePart(field: Expression, source: Expression, child: Expression) From dcaf5b2aa1e66ab3898e87750477bb292883f096 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 18:52:14 +0300 Subject: [PATCH 30/42] Regenerate results of date_part.sql --- sql/core/src/test/resources/sql-tests/results/date_part.sql.out | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 776786850e9d..f519dc1d5b1f 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -400,7 +400,7 @@ select date_part('not_supported', c) from t struct<> -- !query 49 output org.apache.spark.sql.AnalysisException -Literals of type 'not_supported' are currently not supported.;; line 1 pos 7 +Literals of type 'not_supported' are currently not supported for the string type.;; line 1 pos 7 -- !query 50 From f202b154c260775acc60f71d61f46a2095bebb82 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 30 Sep 2019 19:10:48 +0300 Subject: [PATCH 31/42] Add tests for intervals to date_part.sql --- .../resources/sql-tests/inputs/date_part.sql | 75 +++ .../sql-tests/results/date_part.sql.out | 468 +++++++++++++++++- 2 files changed, 542 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql index fd0fb50f7146..a63cdafb745a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/date_part.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/date_part.sql @@ -68,3 +68,78 @@ select date_part('not_supported', c) from t; select date_part(c, c) from t; select date_part(null, c) from t; + +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c; + +select date_part('millennium', c) from t2; +select date_part('millennia', c) from t2; +select date_part('mil', c) from t2; +select date_part('mils', c) from t2; + +select date_part('century', c) from t2; +select date_part('centuries', c) from t2; +select date_part('c', c) from t2; +select date_part('cent', c) from t2; + +select date_part('decade', c) from t2; +select date_part('decades', c) from t2; +select date_part('dec', c) from t2; +select date_part('decs', c) from t2; + +select date_part('year', c) from t2; +select date_part('y', c) from t2; +select date_part('years', c) from t2; +select date_part('yr', c) from t2; +select date_part('yrs', c) from t2; + +select date_part('quarter', c) from t2; +select date_part('qtr', c) from t2; + +select date_part('month', c) from t2; +select date_part('mon', c) from t2; +select date_part('mons', c) from t2; +select date_part('months', c) from t2; + +select date_part('day', c) from t2; +select date_part('d', c) from t2; +select date_part('days', c) from t2; + +select date_part('hour', c) from t2; +select date_part('h', c) from t2; +select date_part('hours', c) from t2; +select date_part('hr', c) from t2; +select date_part('hrs', c) from t2; + +select date_part('minute', c) from t2; +select date_part('m', c) from t2; +select date_part('min', c) from t2; +select date_part('mins', c) from t2; +select date_part('minutes', c) from t2; + +select date_part('second', c) from t2; +select date_part('s', c) from t2; +select date_part('sec', c) from t2; +select date_part('seconds', c) from t2; +select date_part('secs', c) from t2; + +select date_part('milliseconds', c) from t2; +select date_part('msec', c) from t2; +select date_part('msecs', c) from t2; +select date_part('millisecon', c) from t2; +select date_part('mseconds', c) from t2; +select date_part('ms', c) from t2; + +select date_part('microseconds', c) from t2; +select date_part('usec', c) from t2; +select date_part('usecs', c) from t2; +select date_part('useconds', c) from t2; +select date_part('microsecon', c) from t2; +select date_part('us', c) from t2; + +select date_part('epoch', c) from t2; + +select date_part('not_supported', c) from t2; + +select date_part(c, c) from t2; + +select date_part(null, c) from t2; diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index f519dc1d5b1f..d2a2c300b77b 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 52 +-- Number of queries: 110 -- !query 0 @@ -418,3 +418,469 @@ select date_part(null, c) from t struct -- !query 51 output NULL + + +-- !query 52 +CREATE TEMPORARY VIEW t2 AS select interval 1010 year 9 month 8 day 7 hour 6 minute 5 second 4 millisecond 3 microsecond as c +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +select date_part('millennium', c) from t2 +-- !query 53 schema +struct +-- !query 53 output +1 + + +-- !query 54 +select date_part('millennia', c) from t2 +-- !query 54 schema +struct +-- !query 54 output +1 + + +-- !query 55 +select date_part('mil', c) from t2 +-- !query 55 schema +struct +-- !query 55 output +1 + + +-- !query 56 +select date_part('mils', c) from t2 +-- !query 56 schema +struct +-- !query 56 output +1 + + +-- !query 57 +select date_part('century', c) from t2 +-- !query 57 schema +struct +-- !query 57 output +10 + + +-- !query 58 +select date_part('centuries', c) from t2 +-- !query 58 schema +struct +-- !query 58 output +10 + + +-- !query 59 +select date_part('c', c) from t2 +-- !query 59 schema +struct +-- !query 59 output +10 + + +-- !query 60 +select date_part('cent', c) from t2 +-- !query 60 schema +struct +-- !query 60 output +10 + + +-- !query 61 +select date_part('decade', c) from t2 +-- !query 61 schema +struct +-- !query 61 output +101 + + +-- !query 62 +select date_part('decades', c) from t2 +-- !query 62 schema +struct +-- !query 62 output +101 + + +-- !query 63 +select date_part('dec', c) from t2 +-- !query 63 schema +struct +-- !query 63 output +101 + + +-- !query 64 +select date_part('decs', c) from t2 +-- !query 64 schema +struct +-- !query 64 output +101 + + +-- !query 65 +select date_part('year', c) from t2 +-- !query 65 schema +struct +-- !query 65 output +1010 + + +-- !query 66 +select date_part('y', c) from t2 +-- !query 66 schema +struct +-- !query 66 output +1010 + + +-- !query 67 +select date_part('years', c) from t2 +-- !query 67 schema +struct +-- !query 67 output +1010 + + +-- !query 68 +select date_part('yr', c) from t2 +-- !query 68 schema +struct +-- !query 68 output +1010 + + +-- !query 69 +select date_part('yrs', c) from t2 +-- !query 69 schema +struct +-- !query 69 output +1010 + + +-- !query 70 +select date_part('quarter', c) from t2 +-- !query 70 schema +struct +-- !query 70 output +4 + + +-- !query 71 +select date_part('qtr', c) from t2 +-- !query 71 schema +struct +-- !query 71 output +4 + + +-- !query 72 +select date_part('month', c) from t2 +-- !query 72 schema +struct +-- !query 72 output +9 + + +-- !query 73 +select date_part('mon', c) from t2 +-- !query 73 schema +struct +-- !query 73 output +9 + + +-- !query 74 +select date_part('mons', c) from t2 +-- !query 74 schema +struct +-- !query 74 output +9 + + +-- !query 75 +select date_part('months', c) from t2 +-- !query 75 schema +struct +-- !query 75 output +9 + + +-- !query 76 +select date_part('day', c) from t2 +-- !query 76 schema +struct +-- !query 76 output +8 + + +-- !query 77 +select date_part('d', c) from t2 +-- !query 77 schema +struct +-- !query 77 output +8 + + +-- !query 78 +select date_part('days', c) from t2 +-- !query 78 schema +struct +-- !query 78 output +8 + + +-- !query 79 +select date_part('hour', c) from t2 +-- !query 79 schema +struct +-- !query 79 output +7 + + +-- !query 80 +select date_part('h', c) from t2 +-- !query 80 schema +struct +-- !query 80 output +7 + + +-- !query 81 +select date_part('hours', c) from t2 +-- !query 81 schema +struct +-- !query 81 output +7 + + +-- !query 82 +select date_part('hr', c) from t2 +-- !query 82 schema +struct +-- !query 82 output +7 + + +-- !query 83 +select date_part('hrs', c) from t2 +-- !query 83 schema +struct +-- !query 83 output +7 + + +-- !query 84 +select date_part('minute', c) from t2 +-- !query 84 schema +struct +-- !query 84 output +6 + + +-- !query 85 +select date_part('m', c) from t2 +-- !query 85 schema +struct +-- !query 85 output +6 + + +-- !query 86 +select date_part('min', c) from t2 +-- !query 86 schema +struct +-- !query 86 output +6 + + +-- !query 87 +select date_part('mins', c) from t2 +-- !query 87 schema +struct +-- !query 87 output +6 + + +-- !query 88 +select date_part('minutes', c) from t2 +-- !query 88 schema +struct +-- !query 88 output +6 + + +-- !query 89 +select date_part('second', c) from t2 +-- !query 89 schema +struct +-- !query 89 output +5.004003 + + +-- !query 90 +select date_part('s', c) from t2 +-- !query 90 schema +struct +-- !query 90 output +5.004003 + + +-- !query 91 +select date_part('sec', c) from t2 +-- !query 91 schema +struct +-- !query 91 output +5.004003 + + +-- !query 92 +select date_part('seconds', c) from t2 +-- !query 92 schema +struct +-- !query 92 output +5.004003 + + +-- !query 93 +select date_part('secs', c) from t2 +-- !query 93 schema +struct +-- !query 93 output +5.004003 + + +-- !query 94 +select date_part('milliseconds', c) from t2 +-- !query 94 schema +struct +-- !query 94 output +5004.003 + + +-- !query 95 +select date_part('msec', c) from t2 +-- !query 95 schema +struct +-- !query 95 output +5004.003 + + +-- !query 96 +select date_part('msecs', c) from t2 +-- !query 96 schema +struct +-- !query 96 output +5004.003 + + +-- !query 97 +select date_part('millisecon', c) from t2 +-- !query 97 schema +struct +-- !query 97 output +5004.003 + + +-- !query 98 +select date_part('mseconds', c) from t2 +-- !query 98 schema +struct +-- !query 98 output +5004.003 + + +-- !query 99 +select date_part('ms', c) from t2 +-- !query 99 schema +struct +-- !query 99 output +5004.003 + + +-- !query 100 +select date_part('microseconds', c) from t2 +-- !query 100 schema +struct +-- !query 100 output +5004003 + + +-- !query 101 +select date_part('usec', c) from t2 +-- !query 101 schema +struct +-- !query 101 output +5004003 + + +-- !query 102 +select date_part('usecs', c) from t2 +-- !query 102 schema +struct +-- !query 102 output +5004003 + + +-- !query 103 +select date_part('useconds', c) from t2 +-- !query 103 schema +struct +-- !query 103 output +5004003 + + +-- !query 104 +select date_part('microsecon', c) from t2 +-- !query 104 schema +struct +-- !query 104 output +5004003 + + +-- !query 105 +select date_part('us', c) from t2 +-- !query 105 schema +struct +-- !query 105 output +5004003 + + +-- !query 106 +select date_part('epoch', c) from t2 +-- !query 106 schema +struct +-- !query 106 output +31873892788.332003 + + +-- !query 107 +select date_part('not_supported', c) from t2 +-- !query 107 schema +struct<> +-- !query 107 output +org.apache.spark.sql.AnalysisException +Literals of type 'not_supported' are currently not supported for the interval type.;; line 1 pos 7 + + +-- !query 108 +select date_part(c, c) from t2 +-- !query 108 schema +struct<> +-- !query 108 output +org.apache.spark.sql.AnalysisException +The field parameter needs to be a foldable string value.;; line 1 pos 7 + + +-- !query 109 +select date_part(null, c) from t2 +-- !query 109 schema +struct +-- !query 109 output +NULL From dca29e5b435fd6bb0784af2aff0461d4ce2dbd05 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 1 Oct 2019 00:30:53 +0300 Subject: [PATCH 32/42] Remove wrong test --- .../expressions/interval/IntervalExpressionsSuite.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala index 9de38611b1df..58eebb0a4d6f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.interval import scala.language.implicitConversions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{DatePart, ExpressionEvalHelper, Literal} +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval @@ -180,8 +180,4 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Epoch("-1 year"), Decimal(-31557600.0, 18, 6)) checkEvaluation(Epoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) } - - test("date_part from interval") { - checkEvaluation(new DatePart(Literal("months"), Literal(new CalendarInterval(10, 0))), 10) - } } From f8a23856aeeb704518058ccf37a54637d6867072 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 1 Oct 2019 10:29:14 +0300 Subject: [PATCH 33/42] Make Dongjoon and Scala style checker happy --- .../expressions/datetimeExpressions.scala | 1 - .../expressions/intervalExpressions.scala | 75 +++---- .../IntervalExpressionsSuite.scala | 188 ++++++++++++++++++ .../interval/IntervalExpressionsSuite.scala | 183 ----------------- 4 files changed, 226 insertions(+), 221 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 0d518104fb97..2fabe48d71ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -30,7 +30,6 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.expressions.interval.IntervalPart import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 406412eed8d8..724aedd0755f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions.interval +package org.apache.spark.sql.catalyst.expressions import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.catalyst.util.IntervalUtils._ @@ -31,9 +30,7 @@ abstract class IntervalPart( val dataType: DataType, func: CalendarInterval => Any, funcName: String) - extends UnaryExpression - with ExpectsInputTypes - with Serializable { + extends UnaryExpression with ExpectsInputTypes with Serializable { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) override protected def nullSafeEval(interval: Any): Any = { func(interval.asInstanceOf[CalendarInterval]) @@ -44,41 +41,45 @@ abstract class IntervalPart( } } -case class Millenniums(child: Expression) - extends IntervalPart(child, IntegerType, getMillenniums, "getMillenniums") +case class IntervalMillenniums(child: Expression) + extends IntervalPart(child, IntegerType, getMillenniums, "getMillenniums") -case class Centuries(child: Expression) - extends IntervalPart(child, IntegerType, getCenturies, "getCenturies") +case class IntervalCenturies(child: Expression) + extends IntervalPart(child, IntegerType, getCenturies, "getCenturies") -case class Decades(child: Expression) - extends IntervalPart(child, IntegerType, getDecades, "getDecades") +case class IntervalDecades(child: Expression) + extends IntervalPart(child, IntegerType, getDecades, "getDecades") -case class Years(child: Expression) extends IntervalPart(child, IntegerType, getYears, "getYears") +case class IntervalYears(child: Expression) + extends IntervalPart(child, IntegerType, getYears, "getYears") -case class Quarters(child: Expression) - extends IntervalPart(child, ByteType, getQuarters, "getQuarters") +case class IntervalQuarters(child: Expression) + extends IntervalPart(child, ByteType, getQuarters, "getQuarters") -case class Months(child: Expression) extends IntervalPart(child, ByteType, getMonths, "getMonths") +case class IntervalMonths(child: Expression) + extends IntervalPart(child, ByteType, getMonths, "getMonths") -case class Days(child: Expression) extends IntervalPart(child, LongType, getDays, "getDays") +case class IntervalDays(child: Expression) + extends IntervalPart(child, LongType, getDays, "getDays") -case class Hours(child: Expression) extends IntervalPart(child, ByteType, getHours, "getHours") +case class IntervalHours(child: Expression) + extends IntervalPart(child, ByteType, getHours, "getHours") -case class Minutes(child: Expression) - extends IntervalPart(child, ByteType, getMinutes, "getMinutes") +case class IntervalMinutes(child: Expression) + extends IntervalPart(child, ByteType, getMinutes, "getMinutes") -case class Seconds(child: Expression) - extends IntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") +case class IntervalSeconds(child: Expression) + extends IntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") -case class Milliseconds(child: Expression) +case class IntervalMilliseconds(child: Expression) extends IntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") -case class Microseconds(child: Expression) +case class IntervalMicroseconds(child: Expression) extends IntervalPart(child, LongType, getMicroseconds, "getMicroseconds") // Number of seconds in 10000 years is 315576000001 (30 days per one month) // which is 12 digits + 6 digits for the fractional part of seconds. -case class Epoch(child: Expression) +case class IntervalEpoch(child: Expression) extends IntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") object IntervalPart { @@ -87,21 +88,21 @@ object IntervalPart { extractField: String, source: Expression, errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { - case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => Millenniums(source) - case "CENTURY" | "CENTURIES" | "C" | "CENT" => Centuries(source) - case "DECADE" | "DECADES" | "DEC" | "DECS" => Decades(source) - case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => Years(source) - case "QUARTER" | "QTR" => Quarters(source) - case "MONTH" | "MON" | "MONS" | "MONTHS" => Months(source) - case "DAY" | "D" | "DAYS" => Days(source) - case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => Hours(source) - case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => Minutes(source) - case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => Seconds(source) + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => IntervalMillenniums(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => IntervalCenturies(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => IntervalDecades(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => IntervalYears(source) + case "QUARTER" | "QTR" => IntervalQuarters(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => IntervalMonths(source) + case "DAY" | "D" | "DAYS" => IntervalDays(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => IntervalHours(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => IntervalMinutes(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => IntervalSeconds(source) case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => - Milliseconds(source) + IntervalMilliseconds(source) case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => - Microseconds(source) - case "EPOCH" => Epoch(source) + IntervalMicroseconds(source) + case "EPOCH" => IntervalEpoch(source) case _ => errorHandleFunc } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala new file mode 100644 index 000000000000..51019d38d923 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -0,0 +1,188 @@ +/* + * 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 scala.language.implicitConversions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.CalendarInterval + +class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { + implicit def interval(s: String): Literal = { + Literal(CalendarInterval.fromString("interval " + s)) + } + + test("millenniums") { + checkEvaluation(IntervalMillenniums("0 years"), 0) + checkEvaluation(IntervalMillenniums("9999 years"), 9) + checkEvaluation(IntervalMillenniums("1000 years"), 1) + checkEvaluation(IntervalMillenniums("-2000 years"), -2) + // Microseconds part must not be taken into account + checkEvaluation(IntervalMillenniums("999 years 400 days"), 0) + // Millennium must be taken from years and months + checkEvaluation(IntervalMillenniums("999 years 12 months"), 1) + checkEvaluation(IntervalMillenniums("1000 years -1 months"), 0) + } + + test("centuries") { + checkEvaluation(IntervalCenturies("0 years"), 0) + checkEvaluation(IntervalCenturies("9999 years"), 99) + checkEvaluation(IntervalCenturies("1000 years"), 10) + checkEvaluation(IntervalCenturies("-2000 years"), -20) + // Microseconds part must not be taken into account + checkEvaluation(IntervalCenturies("99 years 400 days"), 0) + // Century must be taken from years and months + checkEvaluation(IntervalCenturies("99 years 12 months"), 1) + checkEvaluation(IntervalCenturies("100 years -1 months"), 0) + } + + test("decades") { + checkEvaluation(IntervalDecades("0 years"), 0) + checkEvaluation(IntervalDecades("9999 years"), 999) + checkEvaluation(IntervalDecades("1000 years"), 100) + checkEvaluation(IntervalDecades("-2000 years"), -200) + // Microseconds part must not be taken into account + checkEvaluation(IntervalDecades("9 years 400 days"), 0) + // Decade must be taken from years and months + checkEvaluation(IntervalDecades("9 years 12 months"), 1) + checkEvaluation(IntervalDecades("10 years -1 months"), 0) + } + + test("years") { + checkEvaluation(IntervalYears("0 years"), 0) + checkEvaluation(IntervalYears("9999 years"), 9999) + checkEvaluation(IntervalYears("1000 years"), 1000) + checkEvaluation(IntervalYears("-2000 years"), -2000) + // Microseconds part must not be taken into account + checkEvaluation(IntervalYears("9 years 400 days"), 9) + // Year must be taken from years and months + checkEvaluation(IntervalYears("9 years 12 months"), 10) + checkEvaluation(IntervalYears("10 years -1 months"), 9) + } + + test("quarters") { + checkEvaluation(IntervalQuarters("0 months"), 1.toByte) + checkEvaluation(IntervalQuarters("1 months"), 1.toByte) + checkEvaluation(IntervalQuarters("-1 months"), 1.toByte) + checkEvaluation(IntervalQuarters("2 months"), 1.toByte) + checkEvaluation(IntervalQuarters("-2 months"), 1.toByte) + checkEvaluation(IntervalQuarters("1 years -1 months"), 4.toByte) + checkEvaluation(IntervalQuarters("-1 years 1 months"), -2.toByte) + checkEvaluation(IntervalQuarters("2 years 3 months"), 2.toByte) + checkEvaluation(IntervalQuarters("-2 years -3 months"), 0.toByte) + checkEvaluation(IntervalQuarters("9999 years"), 1.toByte) + } + + test("months") { + checkEvaluation(IntervalMonths("0 year"), 0.toByte) + for (m <- -24 to 24) { + checkEvaluation(IntervalMonths(s"$m months"), (m % 12).toByte) + } + checkEvaluation(IntervalMonths("1 year 10 months"), 10.toByte) + checkEvaluation(IntervalMonths("-2 year -10 months"), -10.toByte) + checkEvaluation(IntervalMonths("9999 years"), 0.toByte) + } + + private val largeInterval: String = "9999 years 11 months " + + "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" + + test("days") { + checkEvaluation(IntervalDays("0 days"), 0L) + checkEvaluation(IntervalDays("1 days 100 seconds"), 1L) + checkEvaluation(IntervalDays("-1 days -100 seconds"), -1L) + checkEvaluation(IntervalDays("-365 days"), -365L) + checkEvaluation(IntervalDays("365 days"), 365L) + // Years and months must not be taken into account + checkEvaluation(IntervalDays("100 year 10 months 5 days"), 5L) + checkEvaluation(IntervalDays(largeInterval), 31L) + } + + test("hours") { + checkEvaluation(IntervalHours("0 hours"), 0.toByte) + checkEvaluation(IntervalHours("1 hour"), 1.toByte) + checkEvaluation(IntervalHours("-1 hour"), -1.toByte) + checkEvaluation(IntervalHours("23 hours"), 23.toByte) + checkEvaluation(IntervalHours("-23 hours"), -23.toByte) + // Years and months must not be taken into account + checkEvaluation(IntervalHours("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(IntervalHours(largeInterval), 11.toByte) + } + + test("minutes") { + checkEvaluation(IntervalMinutes("0 minute"), 0.toByte) + checkEvaluation(IntervalMinutes("1 minute"), 1.toByte) + checkEvaluation(IntervalMinutes("-1 minute"), -1.toByte) + checkEvaluation(IntervalMinutes("59 minute"), 59.toByte) + checkEvaluation(IntervalMinutes("-59 minute"), -59.toByte) + // Years and months must not be taken into account + checkEvaluation(IntervalMinutes("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(IntervalMinutes(largeInterval), 59.toByte) + } + + test("seconds") { + checkEvaluation(IntervalSeconds("0 second"), Decimal(0, 8, 6)) + checkEvaluation(IntervalSeconds("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(IntervalSeconds("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(IntervalSeconds("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(IntervalSeconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + // Years and months must not be taken into account + checkEvaluation(IntervalSeconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(IntervalSeconds(largeInterval), Decimal(59.999999, 8, 6)) + checkEvaluation( + IntervalSeconds("10 seconds 1 milliseconds 1 microseconds"), + Decimal(10001001, 8, 6)) + checkEvaluation(IntervalSeconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) + } + + test("milliseconds") { + checkEvaluation(IntervalMilliseconds("0 milliseconds"), Decimal(0, 8, 3)) + checkEvaluation(IntervalMilliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) + checkEvaluation(IntervalMilliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) + checkEvaluation(IntervalMilliseconds("1 second 999 milliseconds"), Decimal(1999.0, 8, 3)) + checkEvaluation( + IntervalMilliseconds("999 milliseconds 1 microsecond"), + Decimal(999.001, 8, 3)) + checkEvaluation(IntervalMilliseconds("-1 second -999 milliseconds"), Decimal(-1999.0, 8, 3)) + // Years and months must not be taken into account + checkEvaluation(IntervalMilliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) + checkEvaluation(IntervalMilliseconds(largeInterval), Decimal(59999.999, 8, 3)) + } + + test("microseconds") { + checkEvaluation(IntervalMicroseconds("0 microseconds"), 0L) + checkEvaluation(IntervalMicroseconds("1 microseconds"), 1L) + checkEvaluation(IntervalMicroseconds("-1 microseconds"), -1L) + checkEvaluation(IntervalMicroseconds("1 second 999 microseconds"), 1000999L) + checkEvaluation(IntervalMicroseconds("999 milliseconds 1 microseconds"), 999001L) + checkEvaluation(IntervalMicroseconds("-1 second -999 microseconds"), -1000999L) + // Years and months must not be taken into account + checkEvaluation(IntervalMicroseconds("11 year 1 microseconds"), 1L) + checkEvaluation(IntervalMicroseconds(largeInterval), 59999999L) + } + + test("epoch") { + checkEvaluation(IntervalEpoch("0 months"), Decimal(0.0, 18, 6)) + checkEvaluation(IntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(IntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation( + IntervalEpoch("1 second 1 millisecond 1 microsecond"), + Decimal(1.001001, 18, 6)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala deleted file mode 100644 index 58eebb0a4d6f..000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/interval/IntervalExpressionsSuite.scala +++ /dev/null @@ -1,183 +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.interval - -import scala.language.implicitConversions - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} -import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.CalendarInterval - -class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { - implicit def interval(s: String): Literal = { - Literal(CalendarInterval.fromString("interval " + s)) - } - - test("millenniums") { - checkEvaluation(Millenniums("0 years"), 0) - checkEvaluation(Millenniums("9999 years"), 9) - checkEvaluation(Millenniums("1000 years"), 1) - checkEvaluation(Millenniums("-2000 years"), -2) - // Microseconds part must not be taken into account - checkEvaluation(Millenniums("999 years 400 days"), 0) - // Millennium must be taken from years and months - checkEvaluation(Millenniums("999 years 12 months"), 1) - checkEvaluation(Millenniums("1000 years -1 months"), 0) - } - - test("centuries") { - checkEvaluation(Centuries("0 years"), 0) - checkEvaluation(Centuries("9999 years"), 99) - checkEvaluation(Centuries("1000 years"), 10) - checkEvaluation(Centuries("-2000 years"), -20) - // Microseconds part must not be taken into account - checkEvaluation(Centuries("99 years 400 days"), 0) - // Century must be taken from years and months - checkEvaluation(Centuries("99 years 12 months"), 1) - checkEvaluation(Centuries("100 years -1 months"), 0) - } - - test("decades") { - checkEvaluation(Decades("0 years"), 0) - checkEvaluation(Decades("9999 years"), 999) - checkEvaluation(Decades("1000 years"), 100) - checkEvaluation(Decades("-2000 years"), -200) - // Microseconds part must not be taken into account - checkEvaluation(Decades("9 years 400 days"), 0) - // Decade must be taken from years and months - checkEvaluation(Decades("9 years 12 months"), 1) - checkEvaluation(Decades("10 years -1 months"), 0) - } - - test("years") { - checkEvaluation(Years("0 years"), 0) - checkEvaluation(Years("9999 years"), 9999) - checkEvaluation(Years("1000 years"), 1000) - checkEvaluation(Years("-2000 years"), -2000) - // Microseconds part must not be taken into account - checkEvaluation(Years("9 years 400 days"), 9) - // Year must be taken from years and months - checkEvaluation(Years("9 years 12 months"), 10) - checkEvaluation(Years("10 years -1 months"), 9) - } - - test("quarters") { - checkEvaluation(Quarters("0 months"), 1.toByte) - checkEvaluation(Quarters("1 months"), 1.toByte) - checkEvaluation(Quarters("-1 months"), 1.toByte) - checkEvaluation(Quarters("2 months"), 1.toByte) - checkEvaluation(Quarters("-2 months"), 1.toByte) - checkEvaluation(Quarters("1 years -1 months"), 4.toByte) - checkEvaluation(Quarters("-1 years 1 months"), -2.toByte) - checkEvaluation(Quarters("2 years 3 months"), 2.toByte) - checkEvaluation(Quarters("-2 years -3 months"), 0.toByte) - checkEvaluation(Quarters("9999 years"), 1.toByte) - } - - test("months") { - checkEvaluation(Months("0 year"), 0.toByte) - for (m <- -24 to 24) { - checkEvaluation(Months(s"$m months"), (m % 12).toByte) - } - checkEvaluation(Months("1 year 10 months"), 10.toByte) - checkEvaluation(Months("-2 year -10 months"), -10.toByte) - checkEvaluation(Months("9999 years"), 0.toByte) - } - - private val largeInterval: String = "9999 years 11 months " + - "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" - - test("days") { - checkEvaluation(Days("0 days"), 0L) - checkEvaluation(Days("1 days 100 seconds"), 1L) - checkEvaluation(Days("-1 days -100 seconds"), -1L) - checkEvaluation(Days("-365 days"), -365L) - checkEvaluation(Days("365 days"), 365L) - // Years and months must not be taken into account - checkEvaluation(Days("100 year 10 months 5 days"), 5L) - checkEvaluation(Days(largeInterval), 31L) - } - - test("hours") { - checkEvaluation(Hours("0 hours"), 0.toByte) - checkEvaluation(Hours("1 hour"), 1.toByte) - checkEvaluation(Hours("-1 hour"), -1.toByte) - checkEvaluation(Hours("23 hours"), 23.toByte) - checkEvaluation(Hours("-23 hours"), -23.toByte) - // Years and months must not be taken into account - checkEvaluation(Hours("100 year 10 months 10 hours"), 10.toByte) - checkEvaluation(Hours(largeInterval), 11.toByte) - } - - test("minutes") { - checkEvaluation(Minutes("0 minute"), 0.toByte) - checkEvaluation(Minutes("1 minute"), 1.toByte) - checkEvaluation(Minutes("-1 minute"), -1.toByte) - checkEvaluation(Minutes("59 minute"), 59.toByte) - checkEvaluation(Minutes("-59 minute"), -59.toByte) - // Years and months must not be taken into account - checkEvaluation(Minutes("100 year 10 months 10 minutes"), 10.toByte) - checkEvaluation(Minutes(largeInterval), 59.toByte) - } - - test("seconds") { - checkEvaluation(Seconds("0 second"), Decimal(0, 8, 6)) - checkEvaluation(Seconds("1 second"), Decimal(1.0, 8, 6)) - checkEvaluation(Seconds("-1 second"), Decimal(-1.0, 8, 6)) - checkEvaluation(Seconds("1 minute 59 second"), Decimal(59.0, 8, 6)) - checkEvaluation(Seconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) - // Years and months must not be taken into account - checkEvaluation(Seconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) - checkEvaluation(Seconds(largeInterval), Decimal(59.999999, 8, 6)) - checkEvaluation(Seconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) - checkEvaluation(Seconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) - } - - test("milliseconds") { - checkEvaluation(Milliseconds("0 milliseconds"), Decimal(0, 8, 3)) - checkEvaluation(Milliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) - checkEvaluation(Milliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) - checkEvaluation(Milliseconds("1 second 999 milliseconds"), Decimal(1999.0, 8, 3)) - checkEvaluation(Milliseconds("999 milliseconds 1 microsecond"), Decimal(999.001, 8, 3)) - checkEvaluation(Milliseconds("-1 second -999 milliseconds"), Decimal(-1999.0, 8, 3)) - // Years and months must not be taken into account - checkEvaluation(Milliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) - checkEvaluation(Milliseconds(largeInterval), Decimal(59999.999, 8, 3)) - } - - test("microseconds") { - checkEvaluation(Microseconds("0 microseconds"), 0L) - checkEvaluation(Microseconds("1 microseconds"), 1L) - checkEvaluation(Microseconds("-1 microseconds"), -1L) - checkEvaluation(Microseconds("1 second 999 microseconds"), 1000999L) - checkEvaluation(Microseconds("999 milliseconds 1 microseconds"), 999001L) - checkEvaluation(Microseconds("-1 second -999 microseconds"), -1000999L) - // Years and months must not be taken into account - checkEvaluation(Microseconds("11 year 1 microseconds"), 1L) - checkEvaluation(Microseconds(largeInterval), 59999999L) - } - - test("epoch") { - checkEvaluation(Epoch("0 months"), Decimal(0.0, 18, 6)) - checkEvaluation(Epoch("10000 years"), Decimal(315576000000.0, 18, 6)) - checkEvaluation(Epoch("1 year"), Decimal(31557600.0, 18, 6)) - checkEvaluation(Epoch("-1 year"), Decimal(-31557600.0, 18, 6)) - checkEvaluation(Epoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) - } -} From 8a494a237b12f55eb509bf9bbfa13f23e986d0c8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 8 Oct 2019 19:49:38 +0300 Subject: [PATCH 34/42] Improve an example --- .../spark/sql/catalyst/expressions/datetimeExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 2c3679b8c2ae..f9a4f76480a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -2084,7 +2084,7 @@ object DatePart { 1.000001 > SELECT _FUNC_('days', interval 1 year 10 months 5 days); 5 - > SELECT _FUNC_('seconds', interval 30 seconds 1 milliseconds 1 microseconds); + > SELECT _FUNC_('seconds', interval 5 hours 30 seconds 1 milliseconds 1 microseconds); 30.001001 """, since = "3.0.0") From f08531b1685124421935f9a86de611a7b7ff1a51 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 8 Oct 2019 19:50:03 +0300 Subject: [PATCH 35/42] Precise epoch calculation from intervals --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 1 + .../org/apache/spark/sql/catalyst/util/IntervalUtils.scala | 7 ++----- .../src/test/resources/sql-tests/results/date_part.sql.out | 2 +- 3 files changed, 4 insertions(+), 6 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 79fc45ec8947..0a357e756db6 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 @@ -68,6 +68,7 @@ object DateTimeUtils { // 60 * 60 * 24 * 365.2425 = 31556952.0 = 12 * 2629746 final val SECONDS_PER_MONTH: Int = 2629746 final val MILLIS_PER_MONTH: Long = SECONDS_PER_MONTH * MILLIS_PER_SECOND + final val MICROS_PER_MONTH: Long = SECONDS_PER_MONTH * MICROS_PER_SECOND // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 78d188f81f62..9351e538333e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -30,8 +30,6 @@ object IntervalUtils { val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS val DAYS_PER_MONTH: Byte = 30 val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY - /* 365.25 days per year assumes leap year every four years */ - val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -83,9 +81,8 @@ object IntervalUtils { // Returns total number of seconds with microseconds fractional part in the given interval. def getEpoch(interval: CalendarInterval): Decimal = { - var result = interval.microseconds - result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) - result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) + val monthsDurationUs = Math.multiplyExact(interval.months, DateTimeUtils.MICROS_PER_MONTH) + val result = Math.addExact(interval.microseconds, monthsDurationUs) Decimal(result, 18, 6) } } diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 8f4edf196075..3114f93171ab 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -857,7 +857,7 @@ select date_part('epoch', c) from t2 -- !query 106 schema struct -- !query 106 output -31873892788.332003 +31896905999.004003 -- !query 107 From e8a61c8d4db2eaafe68cbf459b5de6f76d0bfac3 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 8 Oct 2019 22:05:35 +0300 Subject: [PATCH 36/42] Fix expected results in IntervalExpressionsSuite --- .../sql/catalyst/expressions/IntervalExpressionsSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 51019d38d923..4fb8735fb07c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -178,9 +178,9 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("epoch") { checkEvaluation(IntervalEpoch("0 months"), Decimal(0.0, 18, 6)) - checkEvaluation(IntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) - checkEvaluation(IntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) - checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation(IntervalEpoch("10000 years"), Decimal(315569520000.0, 18, 6)) + checkEvaluation(IntervalEpoch("1 year"), Decimal(31556952.0, 18, 6)) + checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31556952.0, 18, 6)) checkEvaluation( IntervalEpoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) From f8a45b33e3c385107ad17fbebaf895bbd1f8115a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 12 Oct 2019 11:45:15 +0300 Subject: [PATCH 37/42] Revert "Precise epoch calculation from intervals" This reverts commit f08531b1685124421935f9a86de611a7b7ff1a51. --- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 1 - .../org/apache/spark/sql/catalyst/util/IntervalUtils.scala | 7 +++++-- .../src/test/resources/sql-tests/results/date_part.sql.out | 2 +- 3 files changed, 6 insertions(+), 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 0a357e756db6..79fc45ec8947 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 @@ -68,7 +68,6 @@ object DateTimeUtils { // 60 * 60 * 24 * 365.2425 = 31556952.0 = 12 * 2629746 final val SECONDS_PER_MONTH: Int = 2629746 final val MILLIS_PER_MONTH: Long = SECONDS_PER_MONTH * MILLIS_PER_SECOND - final val MICROS_PER_MONTH: Long = SECONDS_PER_MONTH * MICROS_PER_SECOND // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 9351e538333e..78d188f81f62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -30,6 +30,8 @@ object IntervalUtils { val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS val DAYS_PER_MONTH: Byte = 30 val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + /* 365.25 days per year assumes leap year every four years */ + val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -81,8 +83,9 @@ object IntervalUtils { // Returns total number of seconds with microseconds fractional part in the given interval. def getEpoch(interval: CalendarInterval): Decimal = { - val monthsDurationUs = Math.multiplyExact(interval.months, DateTimeUtils.MICROS_PER_MONTH) - val result = Math.addExact(interval.microseconds, monthsDurationUs) + var result = interval.microseconds + result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) + result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) Decimal(result, 18, 6) } } diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 3114f93171ab..8f4edf196075 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -857,7 +857,7 @@ select date_part('epoch', c) from t2 -- !query 106 schema struct -- !query 106 output -31896905999.004003 +31873892788.332003 -- !query 107 From a496d731a26883ba1e087c0096a93d1a55f1726a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 12 Oct 2019 11:57:13 +0300 Subject: [PATCH 38/42] Precise calculation micros per month --- .../sql/catalyst/util/IntervalUtils.scala | 34 +++++++++++-------- .../sql-tests/results/date_part.sql.out | 2 +- 2 files changed, 21 insertions(+), 15 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 78d188f81f62..02816991a4c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -21,17 +21,24 @@ import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { - val MONTHS_PER_YEAR: Int = 12 - val MONTHS_PER_QUARTER: Byte = 3 - val YEARS_PER_MILLENNIUM: Int = 1000 - val YEARS_PER_CENTURY: Int = 100 - val YEARS_PER_DECADE: Int = 10 - val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS - val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS - val DAYS_PER_MONTH: Byte = 30 - val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY - /* 365.25 days per year assumes leap year every four years */ - val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 + final val MONTHS_PER_YEAR: Int = 12 + final val MONTHS_PER_QUARTER: Byte = 3 + final val YEARS_PER_MILLENNIUM: Int = 1000 + final val YEARS_PER_CENTURY: Int = 100 + final val YEARS_PER_DECADE: Int = 10 + final val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + final val MICROS_PER_MINUTE: Long = { + DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + } + // The average year of the Gregorian calendar 365.2425 days long, see + // https://en.wikipedia.org/wiki/Gregorian_calendar + // Leap year occurs every 4 years, except for years that are divisible by 100 + // and not divisible by 400. So, the mean length of of the Gregorian calendar year is: + // 1 mean year = (365 + 1/4 - 1/100 + 1/400) days = 365.2425 days + // The mean year length in seconds is: + // 60 * 60 * 24 * 365.2425 = 31556952.0 = 12 * 2629746 + final val SECONDS_PER_MONTH: Int = 2629746 + final val MICROS_PER_MONTH: Long = SECONDS_PER_MONTH * DateTimeUtils.MICROS_PER_SECOND def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -83,9 +90,8 @@ object IntervalUtils { // Returns total number of seconds with microseconds fractional part in the given interval. def getEpoch(interval: CalendarInterval): Decimal = { - var result = interval.microseconds - result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) - result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) + val monthsDurationUs = Math.multiplyExact(interval.months, MICROS_PER_MONTH) + val result = Math.addExact(interval.microseconds, monthsDurationUs) Decimal(result, 18, 6) } } diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 8f4edf196075..3114f93171ab 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -857,7 +857,7 @@ select date_part('epoch', c) from t2 -- !query 106 schema struct -- !query 106 output -31873892788.332003 +31896905999.004003 -- !query 107 From 47a0290fd882bad78f6c848a5547d9aba7b75319 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 18 Oct 2019 09:39:46 +0300 Subject: [PATCH 39/42] Revert "Precise calculation micros per month" This reverts commit a496d731a26883ba1e087c0096a93d1a55f1726a. --- .../sql/catalyst/util/IntervalUtils.scala | 34 ++++++++----------- .../sql-tests/results/date_part.sql.out | 2 +- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala index 02816991a4c3..78d188f81f62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/IntervalUtils.scala @@ -21,24 +21,17 @@ import org.apache.spark.sql.types.Decimal import org.apache.spark.unsafe.types.CalendarInterval object IntervalUtils { - final val MONTHS_PER_YEAR: Int = 12 - final val MONTHS_PER_QUARTER: Byte = 3 - final val YEARS_PER_MILLENNIUM: Int = 1000 - final val YEARS_PER_CENTURY: Int = 100 - final val YEARS_PER_DECADE: Int = 10 - final val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS - final val MICROS_PER_MINUTE: Long = { - DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS - } - // The average year of the Gregorian calendar 365.2425 days long, see - // https://en.wikipedia.org/wiki/Gregorian_calendar - // Leap year occurs every 4 years, except for years that are divisible by 100 - // and not divisible by 400. So, the mean length of of the Gregorian calendar year is: - // 1 mean year = (365 + 1/4 - 1/100 + 1/400) days = 365.2425 days - // The mean year length in seconds is: - // 60 * 60 * 24 * 365.2425 = 31556952.0 = 12 * 2629746 - final val SECONDS_PER_MONTH: Int = 2629746 - final val MICROS_PER_MONTH: Long = SECONDS_PER_MONTH * DateTimeUtils.MICROS_PER_SECOND + val MONTHS_PER_YEAR: Int = 12 + val MONTHS_PER_QUARTER: Byte = 3 + val YEARS_PER_MILLENNIUM: Int = 1000 + val YEARS_PER_CENTURY: Int = 100 + val YEARS_PER_DECADE: Int = 10 + val MICROS_PER_HOUR: Long = DateTimeUtils.MILLIS_PER_HOUR * DateTimeUtils.MICROS_PER_MILLIS + val MICROS_PER_MINUTE: Long = DateTimeUtils.MILLIS_PER_MINUTE * DateTimeUtils.MICROS_PER_MILLIS + val DAYS_PER_MONTH: Byte = 30 + val MICROS_PER_MONTH: Long = DAYS_PER_MONTH * DateTimeUtils.SECONDS_PER_DAY + /* 365.25 days per year assumes leap year every four years */ + val MICROS_PER_YEAR: Long = (36525L * DateTimeUtils.MICROS_PER_DAY) / 100 def getYears(interval: CalendarInterval): Int = { interval.months / MONTHS_PER_YEAR @@ -90,8 +83,9 @@ object IntervalUtils { // Returns total number of seconds with microseconds fractional part in the given interval. def getEpoch(interval: CalendarInterval): Decimal = { - val monthsDurationUs = Math.multiplyExact(interval.months, MICROS_PER_MONTH) - val result = Math.addExact(interval.microseconds, monthsDurationUs) + var result = interval.microseconds + result += MICROS_PER_YEAR * (interval.months / MONTHS_PER_YEAR) + result += MICROS_PER_MONTH * (interval.months % MONTHS_PER_YEAR) Decimal(result, 18, 6) } } diff --git a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out index 3114f93171ab..8f4edf196075 100644 --- a/sql/core/src/test/resources/sql-tests/results/date_part.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date_part.sql.out @@ -857,7 +857,7 @@ select date_part('epoch', c) from t2 -- !query 106 schema struct -- !query 106 output -31896905999.004003 +31873892788.332003 -- !query 107 From 2099a919b40376db23a782f4c3720a44aaec1fc7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 18 Oct 2019 09:58:13 +0300 Subject: [PATCH 40/42] Fix expected results in IntervalExpressionsSuite --- .../sql/catalyst/expressions/IntervalExpressionsSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 4fb8735fb07c..51019d38d923 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -178,9 +178,9 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("epoch") { checkEvaluation(IntervalEpoch("0 months"), Decimal(0.0, 18, 6)) - checkEvaluation(IntervalEpoch("10000 years"), Decimal(315569520000.0, 18, 6)) - checkEvaluation(IntervalEpoch("1 year"), Decimal(31556952.0, 18, 6)) - checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31556952.0, 18, 6)) + checkEvaluation(IntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(IntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) checkEvaluation( IntervalEpoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) From d4375b5a938c6a93708059fcf66402d8782e8731 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 18 Oct 2019 10:07:42 +0300 Subject: [PATCH 41/42] Add the Extract prefix to all classes --- .../expressions/datetimeExpressions.scala | 5 +- .../expressions/intervalExpressions.scala | 86 ++++---- .../IntervalExpressionsSuite.scala | 192 +++++++++--------- 3 files changed, 146 insertions(+), 137 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index f9a4f76480a1..df901ddd0dff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -2104,7 +2104,10 @@ case class DatePart(field: Expression, source: Expression, child: Expression) val errMsg = s"Literals of type '$fieldStr' are currently not supported " + s"for the ${source.dataType.catalogString} type." if (source.dataType == CalendarIntervalType) { - IntervalPart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) + ExtractIntervalPart.parseExtractField( + fieldStr, + source, + throw new AnalysisException(errMsg)) } else { DatePart.parseExtractField(fieldStr, source, throw new AnalysisException(errMsg)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 724aedd0755f..4301d956702d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -25,12 +25,14 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval -abstract class IntervalPart( +abstract class ExtractIntervalPart( child: Expression, val dataType: DataType, func: CalendarInterval => Any, funcName: String) - extends UnaryExpression with ExpectsInputTypes with Serializable { + extends UnaryExpression + with ExpectsInputTypes + with Serializable { override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) override protected def nullSafeEval(interval: Any): Any = { func(interval.asInstanceOf[CalendarInterval]) @@ -41,68 +43,68 @@ abstract class IntervalPart( } } -case class IntervalMillenniums(child: Expression) - extends IntervalPart(child, IntegerType, getMillenniums, "getMillenniums") +case class ExtractIntervalMillenniums(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getMillenniums, "getMillenniums") -case class IntervalCenturies(child: Expression) - extends IntervalPart(child, IntegerType, getCenturies, "getCenturies") +case class ExtractIntervalCenturies(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getCenturies, "getCenturies") -case class IntervalDecades(child: Expression) - extends IntervalPart(child, IntegerType, getDecades, "getDecades") +case class ExtractIntervalDecades(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getDecades, "getDecades") -case class IntervalYears(child: Expression) - extends IntervalPart(child, IntegerType, getYears, "getYears") +case class ExtractIntervalYears(child: Expression) + extends ExtractIntervalPart(child, IntegerType, getYears, "getYears") -case class IntervalQuarters(child: Expression) - extends IntervalPart(child, ByteType, getQuarters, "getQuarters") +case class ExtractIntervalQuarters(child: Expression) + extends ExtractIntervalPart(child, ByteType, getQuarters, "getQuarters") -case class IntervalMonths(child: Expression) - extends IntervalPart(child, ByteType, getMonths, "getMonths") +case class ExtractIntervalMonths(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMonths, "getMonths") -case class IntervalDays(child: Expression) - extends IntervalPart(child, LongType, getDays, "getDays") +case class ExtractIntervalDays(child: Expression) + extends ExtractIntervalPart(child, LongType, getDays, "getDays") -case class IntervalHours(child: Expression) - extends IntervalPart(child, ByteType, getHours, "getHours") +case class ExtractIntervalHours(child: Expression) + extends ExtractIntervalPart(child, ByteType, getHours, "getHours") -case class IntervalMinutes(child: Expression) - extends IntervalPart(child, ByteType, getMinutes, "getMinutes") +case class ExtractIntervalMinutes(child: Expression) + extends ExtractIntervalPart(child, ByteType, getMinutes, "getMinutes") -case class IntervalSeconds(child: Expression) - extends IntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") +case class ExtractIntervalSeconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 6), getSeconds, "getSeconds") -case class IntervalMilliseconds(child: Expression) - extends IntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") +case class ExtractIntervalMilliseconds(child: Expression) + extends ExtractIntervalPart(child, DecimalType(8, 3), getMilliseconds, "getMilliseconds") -case class IntervalMicroseconds(child: Expression) - extends IntervalPart(child, LongType, getMicroseconds, "getMicroseconds") +case class ExtractIntervalMicroseconds(child: Expression) + extends ExtractIntervalPart(child, LongType, getMicroseconds, "getMicroseconds") // Number of seconds in 10000 years is 315576000001 (30 days per one month) // which is 12 digits + 6 digits for the fractional part of seconds. -case class IntervalEpoch(child: Expression) - extends IntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") +case class ExtractIntervalEpoch(child: Expression) + extends ExtractIntervalPart(child, DecimalType(18, 6), getEpoch, "getEpoch") -object IntervalPart { +object ExtractIntervalPart { def parseExtractField( extractField: String, source: Expression, errorHandleFunc: => Nothing): Expression = extractField.toUpperCase(Locale.ROOT) match { - case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => IntervalMillenniums(source) - case "CENTURY" | "CENTURIES" | "C" | "CENT" => IntervalCenturies(source) - case "DECADE" | "DECADES" | "DEC" | "DECS" => IntervalDecades(source) - case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => IntervalYears(source) - case "QUARTER" | "QTR" => IntervalQuarters(source) - case "MONTH" | "MON" | "MONS" | "MONTHS" => IntervalMonths(source) - case "DAY" | "D" | "DAYS" => IntervalDays(source) - case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => IntervalHours(source) - case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => IntervalMinutes(source) - case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => IntervalSeconds(source) + case "MILLENNIUM" | "MILLENNIA" | "MIL" | "MILS" => ExtractIntervalMillenniums(source) + case "CENTURY" | "CENTURIES" | "C" | "CENT" => ExtractIntervalCenturies(source) + case "DECADE" | "DECADES" | "DEC" | "DECS" => ExtractIntervalDecades(source) + case "YEAR" | "Y" | "YEARS" | "YR" | "YRS" => ExtractIntervalYears(source) + case "QUARTER" | "QTR" => ExtractIntervalQuarters(source) + case "MONTH" | "MON" | "MONS" | "MONTHS" => ExtractIntervalMonths(source) + case "DAY" | "D" | "DAYS" => ExtractIntervalDays(source) + case "HOUR" | "H" | "HOURS" | "HR" | "HRS" => ExtractIntervalHours(source) + case "MINUTE" | "M" | "MIN" | "MINS" | "MINUTES" => ExtractIntervalMinutes(source) + case "SECOND" | "S" | "SEC" | "SECONDS" | "SECS" => ExtractIntervalSeconds(source) case "MILLISECONDS" | "MSEC" | "MSECS" | "MILLISECON" | "MSECONDS" | "MS" => - IntervalMilliseconds(source) + ExtractIntervalMilliseconds(source) case "MICROSECONDS" | "USEC" | "USECS" | "USECONDS" | "MICROSECON" | "US" => - IntervalMicroseconds(source) - case "EPOCH" => IntervalEpoch(source) + ExtractIntervalMicroseconds(source) + case "EPOCH" => ExtractIntervalEpoch(source) case _ => errorHandleFunc } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala index 51019d38d923..078ec8880021 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala @@ -29,160 +29,164 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("millenniums") { - checkEvaluation(IntervalMillenniums("0 years"), 0) - checkEvaluation(IntervalMillenniums("9999 years"), 9) - checkEvaluation(IntervalMillenniums("1000 years"), 1) - checkEvaluation(IntervalMillenniums("-2000 years"), -2) + checkEvaluation(ExtractIntervalMillenniums("0 years"), 0) + checkEvaluation(ExtractIntervalMillenniums("9999 years"), 9) + checkEvaluation(ExtractIntervalMillenniums("1000 years"), 1) + checkEvaluation(ExtractIntervalMillenniums("-2000 years"), -2) // Microseconds part must not be taken into account - checkEvaluation(IntervalMillenniums("999 years 400 days"), 0) + checkEvaluation(ExtractIntervalMillenniums("999 years 400 days"), 0) // Millennium must be taken from years and months - checkEvaluation(IntervalMillenniums("999 years 12 months"), 1) - checkEvaluation(IntervalMillenniums("1000 years -1 months"), 0) + checkEvaluation(ExtractIntervalMillenniums("999 years 12 months"), 1) + checkEvaluation(ExtractIntervalMillenniums("1000 years -1 months"), 0) } test("centuries") { - checkEvaluation(IntervalCenturies("0 years"), 0) - checkEvaluation(IntervalCenturies("9999 years"), 99) - checkEvaluation(IntervalCenturies("1000 years"), 10) - checkEvaluation(IntervalCenturies("-2000 years"), -20) + checkEvaluation(ExtractIntervalCenturies("0 years"), 0) + checkEvaluation(ExtractIntervalCenturies("9999 years"), 99) + checkEvaluation(ExtractIntervalCenturies("1000 years"), 10) + checkEvaluation(ExtractIntervalCenturies("-2000 years"), -20) // Microseconds part must not be taken into account - checkEvaluation(IntervalCenturies("99 years 400 days"), 0) + checkEvaluation(ExtractIntervalCenturies("99 years 400 days"), 0) // Century must be taken from years and months - checkEvaluation(IntervalCenturies("99 years 12 months"), 1) - checkEvaluation(IntervalCenturies("100 years -1 months"), 0) + checkEvaluation(ExtractIntervalCenturies("99 years 12 months"), 1) + checkEvaluation(ExtractIntervalCenturies("100 years -1 months"), 0) } test("decades") { - checkEvaluation(IntervalDecades("0 years"), 0) - checkEvaluation(IntervalDecades("9999 years"), 999) - checkEvaluation(IntervalDecades("1000 years"), 100) - checkEvaluation(IntervalDecades("-2000 years"), -200) + checkEvaluation(ExtractIntervalDecades("0 years"), 0) + checkEvaluation(ExtractIntervalDecades("9999 years"), 999) + checkEvaluation(ExtractIntervalDecades("1000 years"), 100) + checkEvaluation(ExtractIntervalDecades("-2000 years"), -200) // Microseconds part must not be taken into account - checkEvaluation(IntervalDecades("9 years 400 days"), 0) + checkEvaluation(ExtractIntervalDecades("9 years 400 days"), 0) // Decade must be taken from years and months - checkEvaluation(IntervalDecades("9 years 12 months"), 1) - checkEvaluation(IntervalDecades("10 years -1 months"), 0) + checkEvaluation(ExtractIntervalDecades("9 years 12 months"), 1) + checkEvaluation(ExtractIntervalDecades("10 years -1 months"), 0) } test("years") { - checkEvaluation(IntervalYears("0 years"), 0) - checkEvaluation(IntervalYears("9999 years"), 9999) - checkEvaluation(IntervalYears("1000 years"), 1000) - checkEvaluation(IntervalYears("-2000 years"), -2000) + checkEvaluation(ExtractIntervalYears("0 years"), 0) + checkEvaluation(ExtractIntervalYears("9999 years"), 9999) + checkEvaluation(ExtractIntervalYears("1000 years"), 1000) + checkEvaluation(ExtractIntervalYears("-2000 years"), -2000) // Microseconds part must not be taken into account - checkEvaluation(IntervalYears("9 years 400 days"), 9) + checkEvaluation(ExtractIntervalYears("9 years 400 days"), 9) // Year must be taken from years and months - checkEvaluation(IntervalYears("9 years 12 months"), 10) - checkEvaluation(IntervalYears("10 years -1 months"), 9) + checkEvaluation(ExtractIntervalYears("9 years 12 months"), 10) + checkEvaluation(ExtractIntervalYears("10 years -1 months"), 9) } test("quarters") { - checkEvaluation(IntervalQuarters("0 months"), 1.toByte) - checkEvaluation(IntervalQuarters("1 months"), 1.toByte) - checkEvaluation(IntervalQuarters("-1 months"), 1.toByte) - checkEvaluation(IntervalQuarters("2 months"), 1.toByte) - checkEvaluation(IntervalQuarters("-2 months"), 1.toByte) - checkEvaluation(IntervalQuarters("1 years -1 months"), 4.toByte) - checkEvaluation(IntervalQuarters("-1 years 1 months"), -2.toByte) - checkEvaluation(IntervalQuarters("2 years 3 months"), 2.toByte) - checkEvaluation(IntervalQuarters("-2 years -3 months"), 0.toByte) - checkEvaluation(IntervalQuarters("9999 years"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("0 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 months"), 1.toByte) + checkEvaluation(ExtractIntervalQuarters("1 years -1 months"), 4.toByte) + checkEvaluation(ExtractIntervalQuarters("-1 years 1 months"), -2.toByte) + checkEvaluation(ExtractIntervalQuarters("2 years 3 months"), 2.toByte) + checkEvaluation(ExtractIntervalQuarters("-2 years -3 months"), 0.toByte) + checkEvaluation(ExtractIntervalQuarters("9999 years"), 1.toByte) } test("months") { - checkEvaluation(IntervalMonths("0 year"), 0.toByte) + checkEvaluation(ExtractIntervalMonths("0 year"), 0.toByte) for (m <- -24 to 24) { - checkEvaluation(IntervalMonths(s"$m months"), (m % 12).toByte) + checkEvaluation(ExtractIntervalMonths(s"$m months"), (m % 12).toByte) } - checkEvaluation(IntervalMonths("1 year 10 months"), 10.toByte) - checkEvaluation(IntervalMonths("-2 year -10 months"), -10.toByte) - checkEvaluation(IntervalMonths("9999 years"), 0.toByte) + checkEvaluation(ExtractIntervalMonths("1 year 10 months"), 10.toByte) + checkEvaluation(ExtractIntervalMonths("-2 year -10 months"), -10.toByte) + checkEvaluation(ExtractIntervalMonths("9999 years"), 0.toByte) } private val largeInterval: String = "9999 years 11 months " + "31 days 11 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds" test("days") { - checkEvaluation(IntervalDays("0 days"), 0L) - checkEvaluation(IntervalDays("1 days 100 seconds"), 1L) - checkEvaluation(IntervalDays("-1 days -100 seconds"), -1L) - checkEvaluation(IntervalDays("-365 days"), -365L) - checkEvaluation(IntervalDays("365 days"), 365L) + checkEvaluation(ExtractIntervalDays("0 days"), 0L) + checkEvaluation(ExtractIntervalDays("1 days 100 seconds"), 1L) + checkEvaluation(ExtractIntervalDays("-1 days -100 seconds"), -1L) + checkEvaluation(ExtractIntervalDays("-365 days"), -365L) + checkEvaluation(ExtractIntervalDays("365 days"), 365L) // Years and months must not be taken into account - checkEvaluation(IntervalDays("100 year 10 months 5 days"), 5L) - checkEvaluation(IntervalDays(largeInterval), 31L) + checkEvaluation(ExtractIntervalDays("100 year 10 months 5 days"), 5L) + checkEvaluation(ExtractIntervalDays(largeInterval), 31L) } test("hours") { - checkEvaluation(IntervalHours("0 hours"), 0.toByte) - checkEvaluation(IntervalHours("1 hour"), 1.toByte) - checkEvaluation(IntervalHours("-1 hour"), -1.toByte) - checkEvaluation(IntervalHours("23 hours"), 23.toByte) - checkEvaluation(IntervalHours("-23 hours"), -23.toByte) + checkEvaluation(ExtractIntervalHours("0 hours"), 0.toByte) + checkEvaluation(ExtractIntervalHours("1 hour"), 1.toByte) + checkEvaluation(ExtractIntervalHours("-1 hour"), -1.toByte) + checkEvaluation(ExtractIntervalHours("23 hours"), 23.toByte) + checkEvaluation(ExtractIntervalHours("-23 hours"), -23.toByte) // Years and months must not be taken into account - checkEvaluation(IntervalHours("100 year 10 months 10 hours"), 10.toByte) - checkEvaluation(IntervalHours(largeInterval), 11.toByte) + checkEvaluation(ExtractIntervalHours("100 year 10 months 10 hours"), 10.toByte) + checkEvaluation(ExtractIntervalHours(largeInterval), 11.toByte) } test("minutes") { - checkEvaluation(IntervalMinutes("0 minute"), 0.toByte) - checkEvaluation(IntervalMinutes("1 minute"), 1.toByte) - checkEvaluation(IntervalMinutes("-1 minute"), -1.toByte) - checkEvaluation(IntervalMinutes("59 minute"), 59.toByte) - checkEvaluation(IntervalMinutes("-59 minute"), -59.toByte) + checkEvaluation(ExtractIntervalMinutes("0 minute"), 0.toByte) + checkEvaluation(ExtractIntervalMinutes("1 minute"), 1.toByte) + checkEvaluation(ExtractIntervalMinutes("-1 minute"), -1.toByte) + checkEvaluation(ExtractIntervalMinutes("59 minute"), 59.toByte) + checkEvaluation(ExtractIntervalMinutes("-59 minute"), -59.toByte) // Years and months must not be taken into account - checkEvaluation(IntervalMinutes("100 year 10 months 10 minutes"), 10.toByte) - checkEvaluation(IntervalMinutes(largeInterval), 59.toByte) + checkEvaluation(ExtractIntervalMinutes("100 year 10 months 10 minutes"), 10.toByte) + checkEvaluation(ExtractIntervalMinutes(largeInterval), 59.toByte) } test("seconds") { - checkEvaluation(IntervalSeconds("0 second"), Decimal(0, 8, 6)) - checkEvaluation(IntervalSeconds("1 second"), Decimal(1.0, 8, 6)) - checkEvaluation(IntervalSeconds("-1 second"), Decimal(-1.0, 8, 6)) - checkEvaluation(IntervalSeconds("1 minute 59 second"), Decimal(59.0, 8, 6)) - checkEvaluation(IntervalSeconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("0 second"), Decimal(0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 second"), Decimal(1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-1 second"), Decimal(-1.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("1 minute 59 second"), Decimal(59.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("-59 minutes -59 seconds"), Decimal(-59.0, 8, 6)) // Years and months must not be taken into account - checkEvaluation(IntervalSeconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) - checkEvaluation(IntervalSeconds(largeInterval), Decimal(59.999999, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("100 year 10 months 10 seconds"), Decimal(10.0, 8, 6)) + checkEvaluation(ExtractIntervalSeconds(largeInterval), Decimal(59.999999, 8, 6)) checkEvaluation( - IntervalSeconds("10 seconds 1 milliseconds 1 microseconds"), + ExtractIntervalSeconds("10 seconds 1 milliseconds 1 microseconds"), Decimal(10001001, 8, 6)) - checkEvaluation(IntervalSeconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) + checkEvaluation(ExtractIntervalSeconds("61 seconds 1 microseconds"), Decimal(1000001, 8, 6)) } test("milliseconds") { - checkEvaluation(IntervalMilliseconds("0 milliseconds"), Decimal(0, 8, 3)) - checkEvaluation(IntervalMilliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) - checkEvaluation(IntervalMilliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) - checkEvaluation(IntervalMilliseconds("1 second 999 milliseconds"), Decimal(1999.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("0 milliseconds"), Decimal(0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("1 milliseconds"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("-1 milliseconds"), Decimal(-1.0, 8, 3)) checkEvaluation( - IntervalMilliseconds("999 milliseconds 1 microsecond"), + ExtractIntervalMilliseconds("1 second 999 milliseconds"), + Decimal(1999.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("999 milliseconds 1 microsecond"), Decimal(999.001, 8, 3)) - checkEvaluation(IntervalMilliseconds("-1 second -999 milliseconds"), Decimal(-1999.0, 8, 3)) + checkEvaluation( + ExtractIntervalMilliseconds("-1 second -999 milliseconds"), + Decimal(-1999.0, 8, 3)) // Years and months must not be taken into account - checkEvaluation(IntervalMilliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) - checkEvaluation(IntervalMilliseconds(largeInterval), Decimal(59999.999, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds("100 year 1 millisecond"), Decimal(1.0, 8, 3)) + checkEvaluation(ExtractIntervalMilliseconds(largeInterval), Decimal(59999.999, 8, 3)) } test("microseconds") { - checkEvaluation(IntervalMicroseconds("0 microseconds"), 0L) - checkEvaluation(IntervalMicroseconds("1 microseconds"), 1L) - checkEvaluation(IntervalMicroseconds("-1 microseconds"), -1L) - checkEvaluation(IntervalMicroseconds("1 second 999 microseconds"), 1000999L) - checkEvaluation(IntervalMicroseconds("999 milliseconds 1 microseconds"), 999001L) - checkEvaluation(IntervalMicroseconds("-1 second -999 microseconds"), -1000999L) + checkEvaluation(ExtractIntervalMicroseconds("0 microseconds"), 0L) + checkEvaluation(ExtractIntervalMicroseconds("1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds("-1 microseconds"), -1L) + checkEvaluation(ExtractIntervalMicroseconds("1 second 999 microseconds"), 1000999L) + checkEvaluation(ExtractIntervalMicroseconds("999 milliseconds 1 microseconds"), 999001L) + checkEvaluation(ExtractIntervalMicroseconds("-1 second -999 microseconds"), -1000999L) // Years and months must not be taken into account - checkEvaluation(IntervalMicroseconds("11 year 1 microseconds"), 1L) - checkEvaluation(IntervalMicroseconds(largeInterval), 59999999L) + checkEvaluation(ExtractIntervalMicroseconds("11 year 1 microseconds"), 1L) + checkEvaluation(ExtractIntervalMicroseconds(largeInterval), 59999999L) } test("epoch") { - checkEvaluation(IntervalEpoch("0 months"), Decimal(0.0, 18, 6)) - checkEvaluation(IntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) - checkEvaluation(IntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) - checkEvaluation(IntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("0 months"), Decimal(0.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("10000 years"), Decimal(315576000000.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("1 year"), Decimal(31557600.0, 18, 6)) + checkEvaluation(ExtractIntervalEpoch("-1 year"), Decimal(-31557600.0, 18, 6)) checkEvaluation( - IntervalEpoch("1 second 1 millisecond 1 microsecond"), + ExtractIntervalEpoch("1 second 1 millisecond 1 microsecond"), Decimal(1.001001, 18, 6)) } } From 5620472a26f57590f2d5a67b24b5295980641f1c Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 18 Oct 2019 10:28:09 +0300 Subject: [PATCH 42/42] Change indentation for extends to 2 spaces --- .../sql/catalyst/expressions/intervalExpressions.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala index 4301d956702d..08360c75a474 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala @@ -30,13 +30,14 @@ abstract class ExtractIntervalPart( val dataType: DataType, func: CalendarInterval => Any, funcName: String) - extends UnaryExpression - with ExpectsInputTypes - with Serializable { + extends UnaryExpression with ExpectsInputTypes with Serializable { + override def inputTypes: Seq[AbstractDataType] = Seq(CalendarIntervalType) + override protected def nullSafeEval(interval: Any): Any = { func(interval.asInstanceOf[CalendarInterval]) } + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val iu = IntervalUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, c => s"$iu.$funcName($c)")