Skip to content

Commit 0024da9

Browse files
Davies Liudavies
authored andcommitted
[SQL] address comments for to_date/trunc
This PR address the comments in #7805 cc rxin Author: Davies Liu <[email protected]> Closes #7817 from davies/trunc and squashes the following commits: f729d5f [Davies Liu] rollback cb7f783 [Davies Liu] genCode() is protected 31e52ef [Davies Liu] fix style ed1edc7 [Davies Liu] address comments for #7805
1 parent 27ae851 commit 0024da9

File tree

4 files changed

+14
-11
lines changed

4 files changed

+14
-11
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala

Lines changed: 8 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -726,23 +726,24 @@ case class TruncDate(date: Expression, format: Expression)
726726
override def dataType: DataType = DateType
727727
override def prettyName: String = "trunc"
728728

729-
lazy val minItemConst = DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String])
729+
private lazy val truncLevel: Int =
730+
DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String])
730731

731732
override def eval(input: InternalRow): Any = {
732-
val minItem = if (format.foldable) {
733-
minItemConst
733+
val level = if (format.foldable) {
734+
truncLevel
734735
} else {
735736
DateTimeUtils.parseTruncLevel(format.eval().asInstanceOf[UTF8String])
736737
}
737-
if (minItem == -1) {
738+
if (level == -1) {
738739
// unknown format
739740
null
740741
} else {
741742
val d = date.eval(input)
742743
if (d == null) {
743744
null
744745
} else {
745-
DateTimeUtils.truncDate(d.asInstanceOf[Int], minItem)
746+
DateTimeUtils.truncDate(d.asInstanceOf[Int], level)
746747
}
747748
}
748749
}
@@ -751,7 +752,7 @@ case class TruncDate(date: Expression, format: Expression)
751752
val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
752753

753754
if (format.foldable) {
754-
if (minItemConst == -1) {
755+
if (truncLevel == -1) {
755756
s"""
756757
boolean ${ev.isNull} = true;
757758
${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
@@ -763,7 +764,7 @@ case class TruncDate(date: Expression, format: Expression)
763764
boolean ${ev.isNull} = ${d.isNull};
764765
${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
765766
if (!${ev.isNull}) {
766-
${ev.primitive} = $dtu.truncDate(${d.primitive}, $minItemConst);
767+
${ev.primitive} = $dtu.truncDate(${d.primitive}, $truncLevel);
767768
}
768769
"""
769770
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -794,7 +794,8 @@ object DateTimeUtils {
794794
} else if (level == TRUNC_TO_MONTH) {
795795
d - DateTimeUtils.getDayOfMonth(d) + 1
796796
} else {
797-
throw new Exception(s"Invalid trunc level: $level")
797+
// caller make sure that this should never be reached
798+
sys.error(s"Invalid trunc level: $level")
798799
}
799800
}
800801

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,11 +18,9 @@
1818
package org.apache.spark.sql.catalyst.expressions
1919

2020
import org.scalactic.TripleEqualsSupport.Spread
21-
import org.scalatest.Matchers._
2221

2322
import org.apache.spark.SparkFunSuite
24-
import org.apache.spark.sql.catalyst.InternalRow
25-
import org.apache.spark.sql.catalyst.CatalystTypeConverters
23+
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
2624
import org.apache.spark.sql.catalyst.expressions.codegen._
2725
import org.apache.spark.sql.catalyst.optimizer.DefaultOptimizer
2826
import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project}

sql/core/src/main/scala/org/apache/spark/sql/functions.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2192,6 +2192,9 @@ object functions {
21922192
/**
21932193
* Returns date truncated to the unit specified by the format.
21942194
*
2195+
* @param format: 'year', 'yyyy', 'yy' for truncate by year,
2196+
* or 'month', 'mon', 'mm' for truncate by month
2197+
*
21952198
* @group datetime_funcs
21962199
* @since 1.5.0
21972200
*/

0 commit comments

Comments
 (0)