diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 5f9d11475c94..2fd2d3675661 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1757,7 +1757,8 @@ setMethod("toRadians", #' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. -#' The default format is 'yyyy-MM-dd'. +#' By default, it follows casting rules to a DateType if the format is omitted +#' (equivalent to \code{cast(df$x, "date")}). #' #' @param x Column to parse. #' @param format string to use to parse x Column to DateType. (optional) @@ -1832,10 +1833,11 @@ setMethod("to_json", signature(x = "Column"), #' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. #' If the string cannot be parsed according to the specified format (or default), #' the value of the column will be null. -#' The default format is 'yyyy-MM-dd HH:mm:ss'. +#' By default, it follows casting rules to a TimestampType if the format is omitted +#' (equivalent to \code{cast(df$x, "timestamp")}). #' #' @param x Column to parse. -#' @param format string to use to parse x Column to DateType. (optional) +#' @param format string to use to parse x Column to TimestampType. (optional) #' #' @rdname to_timestamp #' @name to_timestamp diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8b3487c3f108..d9b86aff63fa 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -144,12 +144,6 @@ def _(): 'measured in radians.', } -_functions_2_2 = { - 'to_date': 'Converts a string date into a DateType using the (optionally) specified format.', - 'to_timestamp': 'Converts a string timestamp into a timestamp type using the ' + - '(optionally) specified format.', -} - # math functions that take two arguments as input _binary_mathfunctions = { 'atan2': 'Returns the angle theta from the conversion of rectangular coordinates (x, y) to' + @@ -987,9 +981,10 @@ def months_between(date1, date2): def to_date(col, format=None): """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` - using the optionally specified format. Default format is 'yyyy-MM-dd'. - Specify formats according to + using the optionally specified format. Specify formats according to `SimpleDateFormats `_. + By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format + is omitted (equivalent to ``col.cast("date")``). >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_date(df.t).alias('date')).collect() @@ -1011,9 +1006,10 @@ def to_date(col, format=None): def to_timestamp(col, format=None): """Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or :class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType` - using the optionally specified format. Default format is 'yyyy-MM-dd HH:mm:ss'. Specify - formats according to + using the optionally specified format. Specify formats according to `SimpleDateFormats `_. + By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format + is omitted (equivalent to ``col.cast("timestamp")``). >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) >>> df.select(to_timestamp(df.t).alias('dt')).collect() diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e3fe01eae243..995142d2c583 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -2518,7 +2518,7 @@ def test_datetime_functions(self): from datetime import date, datetime df = self.spark.range(1).selectExpr("'2017-01-22' as dateCol") parse_result = df.select(functions.to_date(functions.col("dateCol"))).first() - self.assertEquals(date(2017, 1, 22), parse_result['to_date(dateCol)']) + self.assertEquals(date(2017, 1, 22), parse_result['to_date(`dateCol`)']) @unittest.skipIf(sys.version_info < (3, 3), "Unittest < 3.3 doesn't support mocking") def test_unbounded_frames(self): 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 a98cd33f2780..de4c94d12abd 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 @@ -1146,44 +1146,21 @@ case class ToUTCTimestamp(left: Expression, right: Expression) } /** - * Returns the date part of a timestamp or string. + * Parses a column to a date based on the given format. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Extracts the date part of the date or timestamp expression `expr`.", + usage = """ + _FUNC_(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to + a date. Returns null with invalid input. By default, it follows casting rules to a date if + the `fmt` is omitted. + """, extended = """ Examples: > SELECT _FUNC_('2009-07-30 04:17:52'); 2009-07-30 - """) -case class ToDate(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { - - // Implicit casting of spark will accept string in both date and timestamp format, as - // well as TimestampType. - override def inputTypes: Seq[AbstractDataType] = Seq(DateType) - - override def dataType: DataType = DateType - - override def eval(input: InternalRow): Any = child.eval(input) - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, d => d) - } - - override def prettyName: String = "to_date" -} - -/** - * Parses a column to a date based on the given format. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.", - extended = """ - Examples: > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 """) -// scalastyle:on line.size.limit case class ParseToDate(left: Expression, format: Option[Expression], child: Expression) extends RuntimeReplaceable { @@ -1194,13 +1171,13 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr def this(left: Expression) = { // backwards compatability - this(left, Option(null), ToDate(left)) + this(left, None, Cast(left, DateType)) } override def flatArguments: Iterator[Any] = Iterator(left, format) override def sql: String = { if (format.isDefined) { - s"$prettyName(${left.sql}, ${format.get.sql}" + s"$prettyName(${left.sql}, ${format.get.sql})" } else { s"$prettyName(${left.sql})" } @@ -1212,24 +1189,36 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr /** * Parses a column to a timestamp based on the supplied format. */ -// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.", + usage = """ + _FUNC_(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to + a timestamp. Returns null with invalid input. By default, it follows casting rules to + a timestamp if the `fmt` is omitted. + """, extended = """ Examples: + > SELECT _FUNC_('2016-12-31 00:12:00'); + 2016-12-31 00:12:00 > SELECT _FUNC_('2016-12-31', 'yyyy-MM-dd'); - 2016-12-31 00:00:00.0 + 2016-12-31 00:00:00 """) -// scalastyle:on line.size.limit -case class ParseToTimestamp(left: Expression, format: Expression, child: Expression) +case class ParseToTimestamp(left: Expression, format: Option[Expression], child: Expression) extends RuntimeReplaceable { def this(left: Expression, format: Expression) = { - this(left, format, Cast(UnixTimestamp(left, format), TimestampType)) + this(left, Option(format), Cast(UnixTimestamp(left, format), TimestampType)) } + def this(left: Expression) = this(left, None, Cast(left, TimestampType)) + override def flatArguments: Iterator[Any] = Iterator(left, format) - override def sql: String = s"$prettyName(${left.sql}, ${format.sql})" + override def sql: String = { + if (format.isDefined) { + s"$prettyName(${left.sql}, ${format.get.sql})" + } else { + s"$prettyName(${left.sql})" + } + } override def prettyName: String = "to_timestamp" override def dataType: DataType = TimestampType 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 bf596fa0a89d..2f262d99539d 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 @@ -427,7 +427,7 @@ object DateTimeUtils { * The return type is [[Option]] in order to distinguish between 0 and null. The following * formats are allowed: * - * `yyyy`, + * `yyyy` * `yyyy-[m]m` * `yyyy-[m]m-[d]d` * `yyyy-[m]m-[d]d ` diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index ca89bf7db0b4..d3bac0a4d277 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -495,14 +495,6 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { NextDay(Literal(Date.valueOf("2015-07-23")), Literal.create(null, StringType)), null) } - test("function to_date") { - checkEvaluation( - ToDate(Literal(Date.valueOf("2015-07-22"))), - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-22"))) - checkEvaluation(ToDate(Literal.create(null, DateType)), null) - checkConsistencyBetweenInterpretedAndCodegen(ToDate, DateType) - } - test("function trunc") { def testTrunc(input: Date, fmt: String, expected: Date): Unit = { checkEvaluation(TruncDate(Literal.create(input, DateType), Literal.create(fmt, StringType)), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 987011edfe1e..4561b307b32d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2683,13 +2683,12 @@ object functions { def unix_timestamp(s: Column, p: String): Column = withExpr { UnixTimestamp(s.expr, Literal(p)) } /** - * Convert time string to a Unix timestamp (in seconds). - * Uses the pattern "yyyy-MM-dd HH:mm:ss" and will return null on failure. + * Convert time string to a Unix timestamp (in seconds) by casting rules to `TimestampType`. * @group datetime_funcs * @since 2.2.0 */ def to_timestamp(s: Column): Column = withExpr { - new ParseToTimestamp(s.expr, Literal("yyyy-MM-dd HH:mm:ss")) + new ParseToTimestamp(s.expr) } /** @@ -2704,15 +2703,15 @@ object functions { } /** - * Converts the column into DateType. + * Converts the column into `DateType` by casting rules to `DateType`. * * @group datetime_funcs * @since 1.5.0 */ - def to_date(e: Column): Column = withExpr { ToDate(e.expr) } + def to_date(e: Column): Column = withExpr { new ParseToDate(e.expr) } /** - * Converts the column into a DateType with a specified format + * Converts the column into a `DateType` with a specified format * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) * return null if fail. * diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 3fd1c37e7179..e957f693a983 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -2,3 +2,7 @@ -- [SPARK-16836] current_date and current_timestamp literals select current_date = current_date(), current_timestamp = current_timestamp(); + +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd'); + +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd'); diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 032e4258500f..13e1e48b038a 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 1 +-- Number of queries: 3 -- !query 0 @@ -8,3 +8,19 @@ select current_date = current_date(), current_timestamp = current_timestamp() struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> -- !query 0 output true true + + +-- !query 1 +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query 1 schema +struct +-- !query 1 output +NULL 2016-12-31 2016-12-31 + + +-- !query 2 +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query 2 schema +struct +-- !query 2 output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 2acda3f00732..3a8694839bb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -387,7 +387,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("to_date(s)"), Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) - // Now with format + // now with format checkAnswer( df.select(to_date(col("t"), "yyyy-MM-dd")), Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), @@ -400,7 +400,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { df.select(to_date(col("s"), "yyyy-MM-dd")), Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) - // now switch format + // now switch format checkAnswer( df.select(to_date(col("s"), "yyyy-dd-MM")), Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31"))))