Repository: spark
Updated Branches:
  refs/heads/branch-2.4 51f3659b7 -> 5d7444996


Revert "[SPARK-23715][SQL] the input of to/from_utc_timestamp can not have 
timezone

## What changes were proposed in this pull request?

This reverts commit 417ad92502e714da71552f64d0e1257d2fd5d3d0.

We decided to keep the current behaviors unchanged and will consider whether we 
will deprecate the  these functions in 3.0. For more details, see the 
discussion in https://issues.apache.org/jira/browse/SPARK-23715

## How was this patch tested?

The existing tests.

Closes #22505 from gatorsmile/revertSpark-23715.

Authored-by: gatorsmile <gatorsm...@gmail.com>
Signed-off-by: Wenchen Fan <wenc...@databricks.com>
(cherry picked from commit 5d25e154408f71d24c4829165a16014fdacdd209)
Signed-off-by: Wenchen Fan <wenc...@databricks.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5d744499
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5d744499
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5d744499

Branch: refs/heads/branch-2.4
Commit: 5d744499667fcd08825bca0ac6d5d90d6e110ebc
Parents: 51f3659
Author: gatorsmile <gatorsm...@gmail.com>
Authored: Fri Sep 21 10:39:45 2018 +0800
Committer: Wenchen Fan <wenc...@databricks.com>
Committed: Fri Sep 21 10:40:13 2018 +0800

----------------------------------------------------------------------
 docs/sql-programming-guide.md                   |   1 -
 .../sql/catalyst/analysis/TypeCoercion.scala    |  30 +----
 .../expressions/datetimeExpressions.scala       |  42 ------
 .../spark/sql/catalyst/util/DateTimeUtils.scala |  22 +--
 .../org/apache/spark/sql/internal/SQLConf.scala |   7 -
 .../catalyst/analysis/TypeCoercionSuite.scala   |  12 +-
 .../resources/sql-tests/inputs/datetime.sql     |  33 -----
 .../sql-tests/results/datetime.sql.out          | 135 +------------------
 .../apache/spark/sql/DateFunctionsSuite.scala   |   9 --
 9 files changed, 13 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/docs/sql-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index d2e3ee3..71ca25a 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -1953,7 +1953,6 @@ working with timestamps in `pandas_udf`s to get the best 
performance, see
   - Since Spark 2.4, renaming a managed table to existing location is not 
allowed. An exception is thrown when attempting to rename a managed table to 
existing location.
   - Since Spark 2.4, the type coercion rules can automatically promote the 
argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest 
common type, no matter how the input arguments order. In prior Spark versions, 
the promotion could fail in some specific orders (e.g., TimestampType, 
IntegerType and StringType) and throw an exception.
   - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in 
addition to the traditional cache invalidation mechanism. The non-cascading 
cache invalidation mechanism allows users to remove a cache without impacting 
its dependent caches. This new cache invalidation mechanism is used in 
scenarios where the data of the cache to be removed is still valid, e.g., 
calling unpersist() on a Dataset, or dropping a temporary view. This allows 
users to free up memory and keep the desired caches valid at the same time.
-  - In version 2.3 and earlier, `to_utc_timestamp` and `from_utc_timestamp` 
respect the timezone in the input timestamp string, which breaks the assumption 
that the input timestamp is in a specific timezone. Therefore, these 2 
functions can return unexpected results. In version 2.4 and later, this problem 
has been fixed. `to_utc_timestamp` and `from_utc_timestamp` will return null if 
the input timestamp string contains timezone. As an example, 
`from_utc_timestamp('2000-10-10 00:00:00', 'GMT+1')` will return `2000-10-10 
01:00:00` in both Spark 2.3 and 2.4. However, `from_utc_timestamp('2000-10-10 
00:00:00+00:00', 'GMT+1')`, assuming a local timezone of GMT+8, will return 
`2000-10-10 09:00:00` in Spark 2.3 but `null` in 2.4. For people who don't care 
about this problem and want to retain the previous behavior to keep their query 
unchanged, you can set `spark.sql.function.rejectTimezoneInString` to false. 
This option will be removed in Spark 3.0 and should only be used as a temporary 
w
 orkaround.
   - In version 2.3 and earlier, Spark converts Parquet Hive tables by default 
but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. 
This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 
'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 
2.4, Spark respects Parquet/ORC specific table properties while converting 
Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS 
PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy 
parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would 
be uncompressed parquet files.
   - Since Spark 2.0, Spark converts Parquet Hive tables by default for better 
performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. 
It means Spark uses its own ORC support by default instead of Hive SerDe. As an 
example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive 
SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC 
data source table and ORC vectorization would be applied. To set `false` to 
`spark.sql.hive.convertMetastoreOrc` restores the previous behavior.
   - In version 2.3 and earlier, CSV rows are considered as malformed if at 
least one column value in the row is malformed. CSV parser dropped such rows in 
the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 
2.4, CSV row is considered as malformed only when it contains malformed column 
values requested from CSV datasource, other values can be ignored. As an 
example, CSV file contains the "id,name" header and one row "1234". In Spark 
2.4, selection of the id column consists of a row with one column value 1234 
but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore 
the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to 
`false`.

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
index 288b635..49d286f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
@@ -60,7 +60,7 @@ object TypeCoercion {
       IfCoercion ::
       StackCoercion ::
       Division ::
-      new ImplicitTypeCasts(conf) ::
+      ImplicitTypeCasts ::
       DateTimeOperations ::
       WindowFrameCoercion ::
       Nil
@@ -841,33 +841,12 @@ object TypeCoercion {
   /**
    * Casts types according to the expected input types for [[Expression]]s.
    */
-  class ImplicitTypeCasts(conf: SQLConf) extends TypeCoercionRule {
-
-    private def rejectTzInString = 
conf.getConf(SQLConf.REJECT_TIMEZONE_IN_STRING)
-
+  object ImplicitTypeCasts extends TypeCoercionRule {
     override protected def coerceTypes(
         plan: LogicalPlan): LogicalPlan = plan resolveExpressions {
       // Skip nodes who's children have not been resolved yet.
       case e if !e.childrenResolved => e
 
-      // Special rules for `from/to_utc_timestamp`. These 2 functions assume 
the input timestamp
-      // string is in a specific timezone, so the string itself should not 
contain timezone.
-      // TODO: We should move the type coercion logic to expressions instead 
of a central
-      // place to put all the rules.
-      case e: FromUTCTimestamp if e.left.dataType == StringType =>
-        if (rejectTzInString) {
-          e.copy(left = StringToTimestampWithoutTimezone(e.left))
-        } else {
-          e.copy(left = Cast(e.left, TimestampType))
-        }
-
-      case e: ToUTCTimestamp if e.left.dataType == StringType =>
-        if (rejectTzInString) {
-          e.copy(left = StringToTimestampWithoutTimezone(e.left))
-        } else {
-          e.copy(left = Cast(e.left, TimestampType))
-        }
-
       case b @ BinaryOperator(left, right) if left.dataType != right.dataType 
=>
         findTightestCommonType(left.dataType, right.dataType).map { commonType 
=>
           if (b.inputType.acceptsType(commonType)) {
@@ -884,7 +863,7 @@ object TypeCoercion {
       case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty =>
         val children: Seq[Expression] = e.children.zip(e.inputTypes).map { 
case (in, expected) =>
           // If we cannot do the implicit cast, just use the original input.
-          ImplicitTypeCasts.implicitCast(in, expected).getOrElse(in)
+          implicitCast(in, expected).getOrElse(in)
         }
         e.withNewChildren(children)
 
@@ -900,9 +879,6 @@ object TypeCoercion {
         }
         e.withNewChildren(children)
     }
-  }
-
-  object ImplicitTypeCasts {
 
     /**
      * Given an expected data type, try to cast the expression and return the 
cast expression.

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.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 f95798d..eb78e39 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
@@ -1018,48 +1018,6 @@ case class TimeAdd(start: Expression, interval: 
Expression, timeZoneId: Option[S
 }
 
 /**
- * A special expression used to convert the string input of 
`to/from_utc_timestamp` to timestamp,
- * which requires the timestamp string to not have timezone information, 
otherwise null is returned.
- */
-case class StringToTimestampWithoutTimezone(child: Expression, timeZoneId: 
Option[String] = None)
-  extends UnaryExpression with TimeZoneAwareExpression with ExpectsInputTypes {
-
-  override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
-    copy(timeZoneId = Option(timeZoneId))
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(StringType)
-  override def dataType: DataType = TimestampType
-  override def nullable: Boolean = true
-  override def toString: String = child.toString
-  override def sql: String = child.sql
-
-  override def nullSafeEval(input: Any): Any = {
-    DateTimeUtils.stringToTimestamp(
-      input.asInstanceOf[UTF8String], timeZone, rejectTzInString = true).orNull
-  }
-
-  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
-    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
-    val tz = ctx.addReferenceObj("timeZone", timeZone)
-    val longOpt = ctx.freshName("longOpt")
-    val eval = child.genCode(ctx)
-    val code = code"""
-       |${eval.code}
-       |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = true;
-       |${CodeGenerator.JAVA_LONG} ${ev.value} = 
${CodeGenerator.defaultValue(TimestampType)};
-       |if (!${eval.isNull}) {
-       |  scala.Option<Long> $longOpt = $dtu.stringToTimestamp(${eval.value}, 
$tz, true);
-       |  if ($longOpt.isDefined()) {
-       |    ${ev.value} = ((Long) $longOpt.get()).longValue();
-       |    ${ev.isNull} = false;
-       |  }
-       |}
-     """.stripMargin
-    ev.copy(code = code)
-  }
-}
-
-/**
  * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in 
UTC, and renders
  * that time as a timestamp in the given time zone. For example, 'GMT+1' would 
yield
  * '2017-07-14 03:40:00.0'.

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
----------------------------------------------------------------------
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 02813d3..81d7274 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
@@ -300,28 +300,10 @@ object DateTimeUtils {
    * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us]+[h]h:[m]m`
    */
   def stringToTimestamp(s: UTF8String): Option[SQLTimestamp] = {
-    stringToTimestamp(s, defaultTimeZone(), rejectTzInString = false)
+    stringToTimestamp(s, defaultTimeZone())
   }
 
   def stringToTimestamp(s: UTF8String, timeZone: TimeZone): 
Option[SQLTimestamp] = {
-    stringToTimestamp(s, timeZone, rejectTzInString = false)
-  }
-
-  /**
-   * Converts a timestamp string to microseconds from the unix epoch, w.r.t. 
the given timezone.
-   * Returns None if the input string is not a valid timestamp format.
-   *
-   * @param s the input timestamp string.
-   * @param timeZone the timezone of the timestamp string, will be ignored if 
the timestamp string
-   *                 already contains timezone information and `forceTimezone` 
is false.
-   * @param rejectTzInString if true, rejects timezone in the input string, 
i.e., if the
-   *                         timestamp string contains timezone, like 
`2000-10-10 00:00:00+00:00`,
-   *                         return None.
-   */
-  def stringToTimestamp(
-      s: UTF8String,
-      timeZone: TimeZone,
-      rejectTzInString: Boolean): Option[SQLTimestamp] = {
     if (s == null) {
       return None
     }
@@ -439,8 +421,6 @@ object DateTimeUtils {
       return None
     }
 
-    if (tz.isDefined && rejectTzInString) return None
-
     val c = if (tz.isEmpty) {
       Calendar.getInstance(timeZone)
     } else {

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 169649c..e63dbba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1388,13 +1388,6 @@ object SQLConf {
       .stringConf
       .createWithDefault("")
 
-  val REJECT_TIMEZONE_IN_STRING = 
buildConf("spark.sql.function.rejectTimezoneInString")
-    .internal()
-    .doc("If true, `to_utc_timestamp` and `from_utc_timestamp` return null if 
the input string " +
-      "contains a timezone part, e.g. `2000-10-10 00:00:00+00:00`.")
-    .booleanConf
-    .createWithDefault(true)
-
   object PartitionOverwriteMode extends Enumeration {
     val STATIC, DYNAMIC = Value
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
index 461eda4..1602f4d 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
@@ -680,11 +680,11 @@ class TypeCoercionSuite extends AnalysisTest {
   test("cast NullType for expressions that implement ExpectsInputTypes") {
     import TypeCoercionSuite._
 
-    ruleTest(new TypeCoercion.ImplicitTypeCasts(conf),
+    ruleTest(TypeCoercion.ImplicitTypeCasts,
       AnyTypeUnaryExpression(Literal.create(null, NullType)),
       AnyTypeUnaryExpression(Literal.create(null, NullType)))
 
-    ruleTest(new TypeCoercion.ImplicitTypeCasts(conf),
+    ruleTest(TypeCoercion.ImplicitTypeCasts,
       NumericTypeUnaryExpression(Literal.create(null, NullType)),
       NumericTypeUnaryExpression(Literal.create(null, DoubleType)))
   }
@@ -692,11 +692,11 @@ class TypeCoercionSuite extends AnalysisTest {
   test("cast NullType for binary operators") {
     import TypeCoercionSuite._
 
-    ruleTest(new TypeCoercion.ImplicitTypeCasts(conf),
+    ruleTest(TypeCoercion.ImplicitTypeCasts,
       AnyTypeBinaryOperator(Literal.create(null, NullType), 
Literal.create(null, NullType)),
       AnyTypeBinaryOperator(Literal.create(null, NullType), 
Literal.create(null, NullType)))
 
-    ruleTest(new TypeCoercion.ImplicitTypeCasts(conf),
+    ruleTest(TypeCoercion.ImplicitTypeCasts,
       NumericTypeBinaryOperator(Literal.create(null, NullType), 
Literal.create(null, NullType)),
       NumericTypeBinaryOperator(Literal.create(null, DoubleType), 
Literal.create(null, DoubleType)))
   }
@@ -976,7 +976,7 @@ class TypeCoercionSuite extends AnalysisTest {
   }
 
   test("type coercion for CaseKeyWhen") {
-    ruleTest(new TypeCoercion.ImplicitTypeCasts(conf),
+    ruleTest(TypeCoercion.ImplicitTypeCasts,
       CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))),
       CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), 
Literal("a")))
     )
@@ -1436,7 +1436,7 @@ class TypeCoercionSuite extends AnalysisTest {
   }
 
   test("SPARK-17117 null type coercion in divide") {
-    val rules = Seq(FunctionArgumentConversion, Division, new 
ImplicitTypeCasts(conf))
+    val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts)
     val nullLit = Literal.create(null, NullType)
     ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), 
Cast(nullLit, DoubleType)))
     ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), 
Cast(1L, DoubleType)))

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
----------------------------------------------------------------------
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 4950a4b..547c2be 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -27,36 +27,3 @@ select current_date = current_date(), current_timestamp = 
current_timestamp(), a
 select a, b from ttf2 order by a, current_date;
 
 select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), 
weekday(null), weekday('1582-10-15 13:10:15');
-
-select from_utc_timestamp('2015-07-24 00:00:00', 'PST');
-
-select from_utc_timestamp('2015-01-24 00:00:00', 'PST');
-
-select from_utc_timestamp(null, 'PST');
-
-select from_utc_timestamp('2015-07-24 00:00:00', null);
-
-select from_utc_timestamp(null, null);
-
-select from_utc_timestamp(cast(0 as timestamp), 'PST');
-
-select from_utc_timestamp(cast('2015-01-24' as date), 'PST');
-
-select to_utc_timestamp('2015-07-24 00:00:00', 'PST');
-
-select to_utc_timestamp('2015-01-24 00:00:00', 'PST');
-
-select to_utc_timestamp(null, 'PST');
-
-select to_utc_timestamp('2015-07-24 00:00:00', null);
-
-select to_utc_timestamp(null, null);
-
-select to_utc_timestamp(cast(0 as timestamp), 'PST');
-
-select to_utc_timestamp(cast('2015-01-24' as date), 'PST');
-
--- SPARK-23715: the input of to/from_utc_timestamp can not have timezone
-select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST');
-
-select to_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST');

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/core/src/test/resources/sql-tests/results/datetime.sql.out
----------------------------------------------------------------------
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 9eede30..4e1cfa6 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: 26
+-- Number of queries: 10
 
 
 -- !query 0
@@ -82,138 +82,9 @@ struct<a:int,b:int>
 1      2
 2      3
 
-
 -- !query 9
 select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), 
weekday(null), weekday('1582-10-15 13:10:15')
--- !query 9 schema
+-- !query 3 schema
 struct<weekday(CAST(2007-02-03 AS DATE)):int,weekday(CAST(2009-07-30 AS 
DATE)):int,weekday(CAST(2017-05-27 AS DATE)):int,weekday(CAST(NULL AS 
DATE)):int,weekday(CAST(1582-10-15 13:10:15 AS DATE)):int>
--- !query 9 output
+-- !query 3 output
 5      3       5       NULL    4
-
-
--- !query 10
-select from_utc_timestamp('2015-07-24 00:00:00', 'PST')
--- !query 10 schema
-struct<from_utc_timestamp(2015-07-24 00:00:00, PST):timestamp>
--- !query 10 output
-2015-07-23 17:00:00
-
-
--- !query 11
-select from_utc_timestamp('2015-01-24 00:00:00', 'PST')
--- !query 11 schema
-struct<from_utc_timestamp(2015-01-24 00:00:00, PST):timestamp>
--- !query 11 output
-2015-01-23 16:00:00
-
-
--- !query 12
-select from_utc_timestamp(null, 'PST')
--- !query 12 schema
-struct<from_utc_timestamp(CAST(NULL AS TIMESTAMP), PST):timestamp>
--- !query 12 output
-NULL
-
-
--- !query 13
-select from_utc_timestamp('2015-07-24 00:00:00', null)
--- !query 13 schema
-struct<from_utc_timestamp(2015-07-24 00:00:00, CAST(NULL AS STRING)):timestamp>
--- !query 13 output
-NULL
-
-
--- !query 14
-select from_utc_timestamp(null, null)
--- !query 14 schema
-struct<from_utc_timestamp(CAST(NULL AS TIMESTAMP), CAST(NULL AS 
STRING)):timestamp>
--- !query 14 output
-NULL
-
-
--- !query 15
-select from_utc_timestamp(cast(0 as timestamp), 'PST')
--- !query 15 schema
-struct<from_utc_timestamp(CAST(0 AS TIMESTAMP), PST):timestamp>
--- !query 15 output
-1969-12-31 08:00:00
-
-
--- !query 16
-select from_utc_timestamp(cast('2015-01-24' as date), 'PST')
--- !query 16 schema
-struct<from_utc_timestamp(CAST(CAST(2015-01-24 AS DATE) AS TIMESTAMP), 
PST):timestamp>
--- !query 16 output
-2015-01-23 16:00:00
-
-
--- !query 17
-select to_utc_timestamp('2015-07-24 00:00:00', 'PST')
--- !query 17 schema
-struct<to_utc_timestamp(2015-07-24 00:00:00, PST):timestamp>
--- !query 17 output
-2015-07-24 07:00:00
-
-
--- !query 18
-select to_utc_timestamp('2015-01-24 00:00:00', 'PST')
--- !query 18 schema
-struct<to_utc_timestamp(2015-01-24 00:00:00, PST):timestamp>
--- !query 18 output
-2015-01-24 08:00:00
-
-
--- !query 19
-select to_utc_timestamp(null, 'PST')
--- !query 19 schema
-struct<to_utc_timestamp(CAST(NULL AS TIMESTAMP), PST):timestamp>
--- !query 19 output
-NULL
-
-
--- !query 20
-select to_utc_timestamp('2015-07-24 00:00:00', null)
--- !query 20 schema
-struct<to_utc_timestamp(2015-07-24 00:00:00, CAST(NULL AS STRING)):timestamp>
--- !query 20 output
-NULL
-
-
--- !query 21
-select to_utc_timestamp(null, null)
--- !query 21 schema
-struct<to_utc_timestamp(CAST(NULL AS TIMESTAMP), CAST(NULL AS 
STRING)):timestamp>
--- !query 21 output
-NULL
-
-
--- !query 22
-select to_utc_timestamp(cast(0 as timestamp), 'PST')
--- !query 22 schema
-struct<to_utc_timestamp(CAST(0 AS TIMESTAMP), PST):timestamp>
--- !query 22 output
-1970-01-01 00:00:00
-
-
--- !query 23
-select to_utc_timestamp(cast('2015-01-24' as date), 'PST')
--- !query 23 schema
-struct<to_utc_timestamp(CAST(CAST(2015-01-24 AS DATE) AS TIMESTAMP), 
PST):timestamp>
--- !query 23 output
-2015-01-24 08:00:00
-
-
--- !query 24
-select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST')
--- !query 24 schema
-struct<from_utc_timestamp(2000-10-10 00:00:00+00:00, PST):timestamp>
--- !query 24 output
-NULL
-
-
--- !query 25
-select to_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST')
--- !query 25 schema
-struct<to_utc_timestamp(2000-10-10 00:00:00+00:00, PST):timestamp>
--- !query 25 output
-NULL

http://git-wip-us.apache.org/repos/asf/spark/blob/5d744499/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
----------------------------------------------------------------------
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 3af80b3..c4ec715 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
@@ -23,7 +23,6 @@ import java.util.Locale
 
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.functions._
-import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSQLContext
 import org.apache.spark.unsafe.types.CalendarInterval
 
@@ -730,12 +729,4 @@ class DateFunctionsSuite extends QueryTest with 
SharedSQLContext {
         Row(Timestamp.valueOf("2015-07-24 07:00:00")),
         Row(Timestamp.valueOf("2015-07-24 22:00:00"))))
   }
-
-  test("SPARK-23715: to/from_utc_timestamp can retain the previous behavior") {
-    withSQLConf(SQLConf.REJECT_TIMEZONE_IN_STRING.key -> "false") {
-      checkAnswer(
-        sql("SELECT from_utc_timestamp('2000-10-10 00:00:00+00:00', 'GMT+1')"),
-        Row(Timestamp.valueOf("2000-10-09 18:00:00")))
-    }
-  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to