[spark] branch master updated: [SPARK-35895][SQL] Support subtracting Intervals from TimestampWithoutTZ

2021-06-26 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 645fb59  [SPARK-35895][SQL] Support subtracting Intervals from 
TimestampWithoutTZ
645fb59 is described below

commit 645fb59652fad5a7b84a691e2446b396cf81048a
Author: Gengliang Wang 
AuthorDate: Sat Jun 26 13:19:00 2021 +0300

[SPARK-35895][SQL] Support subtracting Intervals from TimestampWithoutTZ

### What changes were proposed in this pull request?

Support the following operation:
- TimestampWithoutTZ - Year-Month interval

The following operation is actually supported in 
https://github.com/apache/spark/pull/33076/. This PR is to add end-to-end tests 
for them:
- TimestampWithoutTZ - Calendar interval
- TimestampWithoutTZ - Daytime interval

### Why are the changes needed?

Support subtracting all 3 interval types from a timestamp without time zone

### Does this PR introduce _any_ user-facing change?

No, the timestamp without time zone type is not release yet.

### How was this patch tested?

Unit tests

Closes #33086 from gengliangwang/subtract.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  2 +-
 .../test/resources/sql-tests/inputs/datetime.sql   | 11 
 .../sql-tests/results/ansi/datetime.sql.out| 74 +-
 .../sql-tests/results/datetime-legacy.sql.out  | 74 +-
 .../resources/sql-tests/results/datetime.sql.out   | 74 +-
 5 files changed, 231 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 6737ed5..5de228b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -378,7 +378,7 @@ class Analyzer(override val catalogManager: CatalogManager)
 DatetimeSub(l, r, TimeAdd(Cast(l, TimestampType), UnaryMinus(r, 
f)))
   case (DateType, _: YearMonthIntervalType) =>
 DatetimeSub(l, r, DateAddYMInterval(l, UnaryMinus(r, f)))
-  case (TimestampType, _: YearMonthIntervalType) =>
+  case (TimestampType | TimestampWithoutTZType, _: 
YearMonthIntervalType) =>
 DatetimeSub(l, r, TimestampAddYMInterval(l, UnaryMinus(r, f)))
   case (CalendarIntervalType, CalendarIntervalType) |
(_: DayTimeIntervalType, _: DayTimeIntervalType) => s
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 819bf4b..d68c9ff 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql
@@ -236,3 +236,14 @@ select to_timestamp_ntz('2021-06-25 10:11:12') + interval 
'10-9' year to month;
 select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15' day to hour;
 select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 15:40' day to 
minute;
 select to_timestamp_ntz('2021-06-25 10:11:12') + interval '20 
15:40:32.9989' day to second;
+
+-- TimestampWithoutTZ - Intervals
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval 2 day;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '0-0' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '1-2' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '0 0:0:0' day to 
second;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '0 0:0:0.1' day to 
second;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '10-9' year to month;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '20 15' day to hour;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '20 15:40' day to 
minute;
+select to_timestamp_ntz('2021-06-25 10:11:12') - interval '20 
15:40:32.9989' day to second;
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out 
b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
index 33d041b..08b01ca 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 176
+-- Number of queries: 185
 
 
 -- !

[spark] branch master updated (108635a -> 356aef4)

2021-06-28 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 108635a  Revert "[SPARK-35904][SQL] Collapse above RebalancePartitions"
 add 356aef4  [SPARK-35728][SPARK-35778][SQL][TESTS] Check multiply/divide 
of day-time and year-month interval of any fields by a numeric

No new revisions were added by this update.

Summary of changes:
 .../expressions/IntervalExpressionsSuite.scala | 136 -
 1 file changed, 131 insertions(+), 5 deletions(-)

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



[spark] branch master updated: Revert "[SPARK-33995][SQL] Expose make_interval as a Scala function"

2021-06-29 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 7668226  Revert "[SPARK-33995][SQL] Expose make_interval as a Scala 
function"
7668226 is described below

commit 76682268d746e72f0e8aa4cc64860e0bfd90f1ed
Author: Max Gekk 
AuthorDate: Wed Jun 30 09:26:35 2021 +0300

Revert "[SPARK-33995][SQL] Expose make_interval as a Scala function"

### What changes were proposed in this pull request?
This reverts commit e6753c9402b5c40d9e2af662f28bd4f07a0bae17.

### Why are the changes needed?
The `make_interval` function aims to construct values of the legacy 
interval type `CalendarIntervalType` which will be substituted by ANSI interval 
types (see SPARK-27790). Since the function has not been released yet, it would 
be better to don't expose it via public API at all.

### Does this PR introduce _any_ user-facing change?
Should not since the `make_interval` function has not been released yet.

### How was this patch tested?
By existing test suites, and GA/jenkins builds.
    
Closes #33143 from MaxGekk/revert-make_interval.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/functions.scala | 25 
 .../apache/spark/sql/JavaDateFunctionsSuite.java   | 68 --
 .../org/apache/spark/sql/DateFunctionsSuite.scala  | 40 -
 3 files changed, 133 deletions(-)

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 c446d6b..ecd60ff 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
@@ -2929,31 +2929,6 @@ object functions {
   
//
 
   /**
-   * (Scala-specific) Creates a datetime interval
-   *
-   * @param years Number of years
-   * @param months Number of months
-   * @param weeks Number of weeks
-   * @param days Number of days
-   * @param hours Number of hours
-   * @param mins Number of mins
-   * @param secs Number of secs
-   * @return A datetime interval
-   * @group datetime_funcs
-   * @since 3.2.0
-   */
-  def make_interval(
-  years: Column = lit(0),
-  months: Column = lit(0),
-  weeks: Column = lit(0),
-  days: Column = lit(0),
-  hours: Column = lit(0),
-  mins: Column = lit(0),
-  secs: Column = lit(0)): Column = withExpr {
-MakeInterval(years.expr, months.expr, weeks.expr, days.expr, hours.expr, 
mins.expr, secs.expr)
-  }
-
-  /**
* Returns the date that is `numMonths` after `startDate`.
*
* @param startDate A date, timestamp or string. If a string, the data must 
be in a format that
diff --git 
a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDateFunctionsSuite.java 
b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDateFunctionsSuite.java
deleted file mode 100644
index 2d1de77..000
--- 
a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDateFunctionsSuite.java
+++ /dev/null
@@ -1,68 +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 test.org.apache.spark.sql;
-
-import org.apache.spark.sql.Column;
-import org.apache.spark.sql.Dataset;
-import org.apache.spark.sql.Row;
-import org.apache.spark.sql.RowFactory;
-import org.apache.spark.sql.test.TestSparkSession;
-import org.apache.spark.sql.types.StructType;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.sql.Date;
-import java.util.*;
-
-import static org.apache.spark.sql.types.DataTypes.*;
-import static org.apache.spark.sql.functions.*;
-
-public class JavaDateFunctionsSuite {
-  private transient TestSparkSession spark;
-
-  @Before
-  public void setUp() {
-spark = new TestSparkSession();
-}
-
-  @After
-  public void tearDown() {
-spark.stop();
-spark = null;
-  }
-
-  @Test
-  public v

[spark] branch master updated: [SPARK-35935][SQL] Prevent failure of `MSCK REPAIR TABLE` on table refreshing

2021-06-29 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new d28ca9c  [SPARK-35935][SQL] Prevent failure of `MSCK REPAIR TABLE` on 
table refreshing
d28ca9c is described below

commit d28ca9cc9808828118be64a545c3478160fdc170
Author: Max Gekk 
AuthorDate: Wed Jun 30 09:44:52 2021 +0300

[SPARK-35935][SQL] Prevent failure of `MSCK REPAIR TABLE` on table 
refreshing

### What changes were proposed in this pull request?
In the PR, I propose to catch all non-fatal exceptions coming 
`refreshTable()` at the final stage of table repairing, and output an error 
message instead of failing with an exception.

### Why are the changes needed?
1. The uncaught exceptions from table refreshing might be considered as 
regression comparing to previous Spark versions. Table refreshing was 
introduced by https://github.com/apache/spark/pull/31066.
2. This should improve user experience with Spark SQL. For instance, when 
the `MSCK REPAIR TABLE` is performed in a chain of command in SQL where 
catching exception is difficult or even impossible.

### Does this PR introduce _any_ user-facing change?
Yes. Before the changes the `MSCK REPAIR TABLE` command can fail with the 
exception portrayed in SPARK-35935. After the changes, the same command outputs 
error message, and completes successfully.

### How was this patch tested?
By existing test suites.

Closes #33137 from MaxGekk/msck-repair-catch-except.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../scala/org/apache/spark/sql/execution/command/ddl.scala | 10 +-
 1 file changed, 9 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
index 0876b5f..06c6847 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala
@@ -675,7 +675,15 @@ case class RepairTableCommand(
 // This is always the case for Hive format tables, but is not true for 
Datasource tables created
 // before Spark 2.1 unless they are converted via `msck repair table`.
 spark.sessionState.catalog.alterTable(table.copy(tracksPartitionsInCatalog 
= true))
-spark.catalog.refreshTable(tableIdentWithDB)
+try {
+  spark.catalog.refreshTable(tableIdentWithDB)
+} catch {
+  case NonFatal(e) =>
+logError(s"Cannot refresh the table '$tableIdentWithDB'. A query of 
the table " +
+  "might return wrong result if the table was cached. To avoid such 
issue, you should " +
+  "uncache the table manually via the UNCACHE TABLE command after 
table recovering will " +
+  "complete fully.", e)
+}
 logInfo(s"Recovered all partitions: added ($addedAmount), dropped 
($droppedAmount).")
 Seq.empty[Row]
   }

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



[spark] branch branch-3.1 updated (fe412b6 -> b6e8fab)

2021-06-30 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/spark.git.


from fe412b6  [SPARK-35898][SQL] Fix arrays and maps in RowToColumnConverter
 add b6e8fab  [SPARK-35935][SQL][3.1][3.0] Prevent failure of `MSCK REPAIR 
TABLE` on table refreshing

No new revisions were added by this update.

Summary of changes:
 .../scala/org/apache/spark/sql/execution/command/ddl.scala | 10 +-
 1 file changed, 9 insertions(+), 1 deletion(-)

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



[spark] branch branch-3.0 updated (ab46045 -> 6a1361c)

2021-06-30 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/spark.git.


from ab46045  [SPARK-35886][SQL][3.0] PromotePrecision should not overwrite 
genCodePromotePrecision should not overwrite genCode
 add 6a1361c  [SPARK-35935][SQL][3.1][3.0] Prevent failure of `MSCK REPAIR 
TABLE` on table refreshing

No new revisions were added by this update.

Summary of changes:
 .../scala/org/apache/spark/sql/execution/command/ddl.scala | 10 +-
 1 file changed, 9 insertions(+), 1 deletion(-)

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



[spark] branch master updated: [SPARK-35735][SQL] Take into account day-time interval fields in cast

2021-06-30 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 2febd5c  [SPARK-35735][SQL] Take into account day-time interval fields 
in cast
2febd5c is described below

commit 2febd5c3f0c3a0c6660cfb340eb65316a1ca4acd
Author: Angerszh 
AuthorDate: Wed Jun 30 16:05:04 2021 +0300

[SPARK-35735][SQL] Take into account day-time interval fields in cast

### What changes were proposed in this pull request?
Support take into account day-time interval field in cast.

### Why are the changes needed?
To conform to the SQL standard.

### Does this PR introduce _any_ user-facing change?
An user can use `cast(str, DayTimeInterval(DAY, HOUR))`, for instance.

### How was this patch tested?
Added UT.

Closes #32943 from AngersZh/SPARK-35735.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 203 +++--
 .../sql/catalyst/expressions/CastSuiteBase.scala   | 145 ++-
 2 files changed, 324 insertions(+), 24 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 7a6de7f..30a2fa5 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
@@ -119,18 +119,27 @@ object IntervalUtils {
 }
   }
 
+  val supportedFormat = Map(
+(YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO 
MONTH"),
+(YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
+(YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH"),
+(DT.DAY, DT.DAY) -> Seq("[+|-]d", "INTERVAL [+|-]'[+|-]d' DAY"),
+(DT.DAY, DT.HOUR) -> Seq("[+|-]d h", "INTERVAL [+|-]'[+|-]d h' DAY TO 
HOUR"),
+(DT.DAY, DT.MINUTE) -> Seq("[+|-]d h:m", "INTERVAL [+|-]'[+|-]d h:m' DAY 
TO MINUTE"),
+(DT.DAY, DT.SECOND) -> Seq("[+|-]d h:m:s.n", "INTERVAL [+|-]'[+|-]d 
h:m:s.n' DAY TO SECOND"),
+(DT.HOUR, DT.HOUR) -> Seq("[+|-]h", "INTERVAL [+|-]'[+|-]h' HOUR"),
+(DT.HOUR, DT.MINUTE) -> Seq("[+|-]h:m", "INTERVAL [+|-]'[+|-]h:m' HOUR TO 
MINUTE"),
+(DT.HOUR, DT.SECOND) -> Seq("[+|-]h:m:s.n", "INTERVAL [+|-]'[+|-]h:m:s.n' 
HOUR TO SECOND"),
+(DT.MINUTE, DT.MINUTE) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MINUTE"),
+(DT.MINUTE, DT.SECOND) -> Seq("[+|-]m:s.n", "INTERVAL [+|-]'[+|-]m:s.n' 
MINUTE TO SECOND"),
+(DT.SECOND, DT.SECOND) -> Seq("[+|-]s.n", "INTERVAL [+|-]'[+|-]s.n' 
SECOND")
+  )
+
   def castStringToYMInterval(
   input: UTF8String,
   startField: Byte,
   endField: Byte): Int = {
 
-val supportedFormat = Map(
-  (YM.YEAR, YM.MONTH) ->
-Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH"),
-  (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
-  (YM.MONTH, YM.MONTH) -> Seq("[+|-]m", "INTERVAL [+|-]'[+|-]m' MONTH")
-)
-
 def checkStringIntervalType(targetStartField: Byte, targetEndField: Byte): 
Unit = {
   if (startField != targetStartField || endField != targetEndField) {
 throw new IllegalArgumentException(s"Interval string does not match 
year-month format of " +
@@ -151,7 +160,7 @@ object IntervalUtils {
 checkStringIntervalType(YM.YEAR, YM.MONTH)
 toYMInterval(year, month, getSign(firstSign, secondSign))
   case yearMonthIndividualRegex(secondSign, value) =>
-safeToYMInterval {
+safeToInterval {
   val sign = getSign("+", secondSign)
   if (endField == YM.YEAR) {
 sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
@@ -166,7 +175,7 @@ object IntervalUtils {
   }
 }
   case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, 
suffix) =>
-safeToYMInterval {
+safeToInterval {
   val sign = getSign(firstSign, secondSign)
   if ("YEAR".equalsIgnoreCase(suffix)) {
 checkStringIntervalType(YM.YEAR, YM.YEAR)
@@ -202,7 +211,7 @@ object IntervalUtils {
 }
   }
 
-  private def safeToYMInterval(f: =

[spark] branch master updated: [SPARK-35975][SQL] New configuration `spark.sql.timestampType` for the default timestamp type

2021-07-01 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new a643076  [SPARK-35975][SQL] New configuration 
`spark.sql.timestampType` for the default timestamp type
a643076 is described below

commit a643076d4ef622eac505ebf22c9aa2cc909320ac
Author: Gengliang Wang 
AuthorDate: Thu Jul 1 23:25:18 2021 +0300

[SPARK-35975][SQL] New configuration `spark.sql.timestampType` for the 
default timestamp type

### What changes were proposed in this pull request?

Add a new configuration `spark.sql.timestampType`, which configures the 
default timestamp type of Spark SQL, including SQL DDL and Cast clause. Setting 
the configuration as `TIMESTAMP_NTZ` will use `TIMESTAMP WITHOUT TIME ZONE` as 
the default type while putting it as `TIMESTAMP_LTZ` will use `TIMESTAMP WITH 
LOCAL TIME ZONE`.

The default value of the new configuration is TIMESTAMP_LTZ, which is 
consistent with previous Spark releases.

### Why are the changes needed?

A new configuration for switching the default timestamp type as timestamp 
without time zone.

### Does this PR introduce _any_ user-facing change?

No, it's a new feature.

### How was this patch tested?

Unit test

Closes #33176 from gengliangwang/newTsTypeConf.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/parser/AstBuilder.scala |  2 +-
 .../org/apache/spark/sql/internal/SQLConf.scala| 28 ++
 .../sql/catalyst/parser/DataTypeParserSuite.scala  | 14 ++-
 3 files changed, 42 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 224c2d0..361ecc1 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2502,7 +2502,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   case ("float" | "real", Nil) => FloatType
   case ("double", Nil) => DoubleType
   case ("date", Nil) => DateType
-  case ("timestamp", Nil) => TimestampType
+  case ("timestamp", Nil) => SQLConf.get.timestampType
   case ("string", Nil) => StringType
   case ("character" | "char", length :: Nil) => 
CharType(length.getText.toInt)
   case ("varchar", length :: Nil) => VarcharType(length.getText.toInt)
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 30e5a16..3aed3c2 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
@@ -44,6 +44,7 @@ import 
org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import 
org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
+import org.apache.spark.sql.types.{AtomicType, TimestampNTZType, TimestampType}
 import org.apache.spark.unsafe.array.ByteArrayMethods
 import org.apache.spark.util.Utils
 
@@ -2820,6 +2821,24 @@ object SQLConf {
   .booleanConf
   .createWithDefault(true)
 
+  object TimestampTypes extends Enumeration {
+val TIMESTAMP_NTZ, TIMESTAMP_LTZ = Value
+  }
+
+  val TIMESTAMP_TYPE =
+buildConf("spark.sql.timestampType")
+  .doc("Configures the default timestamp type of Spark SQL, including SQL 
DDL and Cast " +
+s"clause. Setting the configuration as 
${TimestampTypes.TIMESTAMP_NTZ.toString} will " +
+"use TIMESTAMP WITHOUT TIME ZONE as the default type while putting it 
as " +
+s"${TimestampTypes.TIMESTAMP_LTZ.toString} will use TIMESTAMP WITH 
LOCAL TIME ZONE. " +
+"Before the 3.2.0 release, Spark only supports the TIMESTAMP WITH " +
+"LOCAL TIME ZONE type.")
+  .version("3.2.0")
+  .stringConf
+  .transform(_.toUpperCase(Locale.ROOT))
+  .checkValues(TimestampTypes.values.map(_.toString))
+  .createWithDefault(TimestampTypes.TIMESTAMP_LTZ.toString)
+
   val DATETIME_JAVA8API_ENABLED = 
buildConf("spark.sql.datetime.java8API.enabled")
 .doc("If the configuration property is set to true, java.time.Instant and 
" +
   "java.time.LocalDate classes of Java 8

[spark] branch master updated: [SPARK-35982][SQL] Allow from_json/to_json for map types where value types are year-month intervals

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 6474226  [SPARK-35982][SQL] Allow from_json/to_json for map types 
where value types are year-month intervals
6474226 is described below

commit 647422685292cd1a46766afa9b07b6fcfc181bbd
Author: Kousuke Saruta 
AuthorDate: Mon Jul 5 10:35:50 2021 +0300

[SPARK-35982][SQL] Allow from_json/to_json for map types where value types 
are year-month intervals

### What changes were proposed in this pull request?

This PR fixes two issues. One is that `to_json` doesn't support `map` types 
where value types are `year-month` interval types like:
```
spark-sql> select to_json(map('a', interval '1-2' year to  month));
21/07/02 11:38:15 ERROR SparkSQLDriver: Failed in [select to_json(map('a', 
interval '1-2' year to  month))]
java.lang.RuntimeException: Failed to convert value 14 (class of class 
java.lang.Integer) with the type of YearMonthIntervalType(0,1) to JSON.
```
The other issue is that even if the issue of `to_json` is resolved, 
`from_json` doesn't support to convert `year-month` interval string to JSON. So 
the result of following query will be `null`.
```
spark-sql> select from_json(to_json(map('a', interval '1-2' year to 
month)), 'a interval year to month');
{"a":null}
```

### Why are the changes needed?

There should be no reason why year-month intervals cannot used as map value 
types.
`CalendarIntervalTypes` can do it.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33181 from sarutak/map-json-yminterval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/json/JacksonGenerator.scala |  9 
 .../spark/sql/catalyst/json/JacksonParser.scala|  7 ++
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 26 ++
 3 files changed, 42 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
index 2567438..9777d56 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
@@ -147,6 +147,15 @@ private[sql] class JacksonGenerator(
   (row: SpecializedGetters, ordinal: Int) =>
 gen.writeString(row.getInterval(ordinal).toString)
 
+case YearMonthIntervalType(start, end) =>
+  (row: SpecializedGetters, ordinal: Int) =>
+val ymString = IntervalUtils.toYearMonthIntervalString(
+  row.getInt(ordinal),
+  IntervalStringStyles.ANSI_STYLE,
+  start,
+  end)
+gen.writeString(ymString)
+
 case BinaryType =>
   (row: SpecializedGetters, ordinal: Int) =>
 gen.writeBinary(row.getBinary(ordinal))
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
index 27e1411..2aa735d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
@@ -295,6 +295,13 @@ class JacksonParser(
   
IntervalUtils.safeStringToInterval(UTF8String.fromString(parser.getText))
   }
 
+case ym: YearMonthIntervalType => (parser: JsonParser) =>
+  parseJsonToken[Integer](parser, dataType) {
+case VALUE_STRING =>
+  val expr = Cast(Literal(parser.getText), ym)
+  Integer.valueOf(expr.eval(EmptyRow).asInstanceOf[Int])
+  }
+
 case st: StructType =>
   val fieldConverters = st.map(_.dataType).map(makeConverter).toArray
   (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index 5485cc1..c2bea8c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.text.SimpleDateFormat
+import java.time.Period
 import java.util.Locale
 
 import collection.JavaConverters._
@@ -27,6 +28,7 @@ import org.apache.spark.sql.functions._
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.SharedSparkSession
 import org.apach

[spark] branch branch-3.2 updated: [SPARK-35982][SQL] Allow from_json/to_json for map types where value types are year-month intervals

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 26bcf02  [SPARK-35982][SQL] Allow from_json/to_json for map types 
where value types are year-month intervals
26bcf02 is described below

commit 26bcf028339c02ca75af31ab8105f7dbe58c49a9
Author: Kousuke Saruta 
AuthorDate: Mon Jul 5 10:35:50 2021 +0300

[SPARK-35982][SQL] Allow from_json/to_json for map types where value types 
are year-month intervals

### What changes were proposed in this pull request?

This PR fixes two issues. One is that `to_json` doesn't support `map` types 
where value types are `year-month` interval types like:
```
spark-sql> select to_json(map('a', interval '1-2' year to  month));
21/07/02 11:38:15 ERROR SparkSQLDriver: Failed in [select to_json(map('a', 
interval '1-2' year to  month))]
java.lang.RuntimeException: Failed to convert value 14 (class of class 
java.lang.Integer) with the type of YearMonthIntervalType(0,1) to JSON.
```
The other issue is that even if the issue of `to_json` is resolved, 
`from_json` doesn't support to convert `year-month` interval string to JSON. So 
the result of following query will be `null`.
```
spark-sql> select from_json(to_json(map('a', interval '1-2' year to 
month)), 'a interval year to month');
{"a":null}
```

### Why are the changes needed?

There should be no reason why year-month intervals cannot used as map value 
types.
`CalendarIntervalTypes` can do it.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33181 from sarutak/map-json-yminterval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit 647422685292cd1a46766afa9b07b6fcfc181bbd)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/json/JacksonGenerator.scala |  9 
 .../spark/sql/catalyst/json/JacksonParser.scala|  7 ++
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 26 ++
 3 files changed, 42 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
index 2567438..9777d56 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
@@ -147,6 +147,15 @@ private[sql] class JacksonGenerator(
   (row: SpecializedGetters, ordinal: Int) =>
 gen.writeString(row.getInterval(ordinal).toString)
 
+case YearMonthIntervalType(start, end) =>
+  (row: SpecializedGetters, ordinal: Int) =>
+val ymString = IntervalUtils.toYearMonthIntervalString(
+  row.getInt(ordinal),
+  IntervalStringStyles.ANSI_STYLE,
+  start,
+  end)
+gen.writeString(ymString)
+
 case BinaryType =>
   (row: SpecializedGetters, ordinal: Int) =>
 gen.writeBinary(row.getBinary(ordinal))
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
index 27e1411..2aa735d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
@@ -295,6 +295,13 @@ class JacksonParser(
   
IntervalUtils.safeStringToInterval(UTF8String.fromString(parser.getText))
   }
 
+case ym: YearMonthIntervalType => (parser: JsonParser) =>
+  parseJsonToken[Integer](parser, dataType) {
+case VALUE_STRING =>
+  val expr = Cast(Literal(parser.getText), ym)
+  Integer.valueOf(expr.eval(EmptyRow).asInstanceOf[Int])
+  }
+
 case st: StructType =>
   val fieldConverters = st.map(_.dataType).map(makeConverter).toArray
   (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index 5485cc1..c2bea8c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.text.SimpleDateFormat
+import java.time.Period
 import java.util.Locale
 
 import collection.JavaConverters._
@@ -27,6 +28,7 @@ import org.apache.sp

[spark] branch master updated (2fe6c94 -> f4237af)

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 2fe6c94  [SPARK-33996][BUILD][FOLLOW-UP] Match SBT's plugin checkstyle 
version to Maven's
 add f4237af  [SPARK-35998][SQL] Make from_csv/to_csv to handle year-month 
intervals properly

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/csv/UnivocityGenerator.scala  |  7 +-
 .../spark/sql/catalyst/csv/UnivocityParser.scala   |  7 +-
 .../org/apache/spark/sql/CsvFunctionsSuite.scala   | 29 ++
 3 files changed, 41 insertions(+), 2 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-35998][SQL] Make from_csv/to_csv to handle year-month intervals properly

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 544b7e1  [SPARK-35998][SQL] Make from_csv/to_csv to handle year-month 
intervals properly
544b7e1 is described below

commit 544b7e16acf51b7c2a9555fb4ebe7b19a00e
Author: Kousuke Saruta 
AuthorDate: Mon Jul 5 13:10:50 2021 +0300

[SPARK-35998][SQL] Make from_csv/to_csv to handle year-month intervals 
properly

### What changes were proposed in this pull request?

This PR fixes an issue that `from_csv/to_csv` doesn't handle year-month 
intervals properly.
`from_csv` throws exception if year-month interval types are given.
```
spark-sql> select from_csv("interval '1-2' year to month", "a interval year 
to month");
21/07/03 04:32:24 ERROR SparkSQLDriver: Failed in [select 
from_csv("interval '1-2' year to month", "a interval year to month")]
java.lang.Exception: Unsupported type: interval year to month
at 
org.apache.spark.sql.errors.QueryExecutionErrors$.unsupportedTypeError(QueryExecutionErrors.scala:775)
at 
org.apache.spark.sql.catalyst.csv.UnivocityParser.makeConverter(UnivocityParser.scala:224)
at 
org.apache.spark.sql.catalyst.csv.UnivocityParser.$anonfun$valueConverters$1(UnivocityParser.scala:134)
```

Also, `to_csv` doesn't handle year-month interval types properly though any 
exception is thrown.
The result of `to_csv` for year-month interval types is not ANSI interval 
compliant form.

```
spark-sql> select to_csv(named_struct("a", interval '1-2' year to month));
14
```
The result above should be `INTERVAL '1-2' YEAR TO MONTH`.

### Why are the changes needed?

Bug fix.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33210 from sarutak/csv-yminterval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit f4237aff7ebece0b8d61e680ecbe759850f25af8)
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/csv/UnivocityGenerator.scala  |  7 +-
 .../spark/sql/catalyst/csv/UnivocityParser.scala   |  7 +-
 .../org/apache/spark/sql/CsvFunctionsSuite.scala   | 29 ++
 3 files changed, 41 insertions(+), 2 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
index 11b31ce..5d70ccb 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala
@@ -22,7 +22,7 @@ import java.io.Writer
 import com.univocity.parsers.csv.CsvWriter
 
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter}
+import org.apache.spark.sql.catalyst.util.{DateFormatter, 
IntervalStringStyles, IntervalUtils, TimestampFormatter}
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.types._
 
@@ -61,6 +61,11 @@ class UnivocityGenerator(
 case TimestampType =>
   (row: InternalRow, ordinal: Int) => 
timestampFormatter.format(row.getLong(ordinal))
 
+case YearMonthIntervalType(start, end) =>
+  (row: InternalRow, ordinal: Int) =>
+IntervalUtils.toYearMonthIntervalString(
+  row.getInt(ordinal), IntervalStringStyles.ANSI_STYLE, start, end)
+
 case udt: UserDefinedType[_] => makeConverter(udt.sqlType)
 
 case dt: DataType =>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
index 672d133..3ec1ea0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala
@@ -26,7 +26,7 @@ import com.univocity.parsers.csv.CsvParser
 import org.apache.spark.SparkUpgradeException
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.{InternalRow, NoopFilters, OrderedFilters}
-import org.apache.spark.sql.catalyst.expressions.{ExprUtils, 
GenericInternalRow}
+import org.apache.spark.sql.catalyst.expressions.{Cast, EmptyRow, ExprUtils, 
GenericInternalRow, Literal}
 import org.apache.spark.sql.catalyst.util._
 import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT
 import org.apache.spark.sql.errors.QueryExecutionErrors
@@ -217,6 +217,11

[spark] branch master updated: [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 5f44acf  [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ
5f44acf is described below

commit 5f44acff3df51721fe891ea50c0a5bcf3a37a719
Author: Gengliang Wang 
AuthorDate: Mon Jul 5 22:30:44 2021 +0300

[SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ

### What changes were proposed in this pull request?

Support new keyword TIMESTAMP_NTZ, which can be used for:

- timestamp without time zone data type in DDL
- timestamp without time zone data type in Cast clause.
- timestamp without time zone data type literal

### Why are the changes needed?

Users can use `TIMESTAMP_NTZ` in DDL/Cast/Literals for the timestamp 
without time zone type directly.

### Does this PR introduce _any_ user-facing change?

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33221 from gengliangwang/timstamp_ntz.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
---
 .../main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala  | 4 
 .../org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala| 1 +
 .../org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala  | 3 +++
 3 files changed, 8 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 5b9107f..c650cf0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2125,6 +2125,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   val zoneId = getZoneId(conf.sessionLocalTimeZone)
   val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, 
DateType))
   specialDate.getOrElse(toLiteral(stringToDate, DateType))
+case "TIMESTAMP_NTZ" =>
+  val specialTs = convertSpecialTimestampNTZ(value).map(Literal(_, 
TimestampNTZType))
+  specialTs.getOrElse(toLiteral(stringToTimestampWithoutTimeZone, 
TimestampNTZType))
 case "TIMESTAMP" =>
   def constructTimestampLTZLiteral(value: String): Literal = {
 val zoneId = getZoneId(conf.sessionLocalTimeZone)
@@ -2525,6 +2528,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   case ("double", Nil) => DoubleType
   case ("date", Nil) => DateType
   case ("timestamp", Nil) => SQLConf.get.timestampType
+  case ("timestamp_ntz", Nil) => TimestampNTZType
   case ("string", Nil) => StringType
   case ("character" | "char", length :: Nil) => 
CharType(length.getText.toInt)
   case ("varchar", length :: Nil) => VarcharType(length.getText.toInt)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
index a6b78e0..d34 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
@@ -58,6 +58,7 @@ class DataTypeParserSuite extends SparkFunSuite with 
SQLHelper {
   checkDataType("deC", DecimalType.USER_DEFAULT)
   checkDataType("DATE", DateType)
   checkDataType("timestamp", TimestampType)
+  checkDataType("timestamp_ntz", TimestampNTZType)
   checkDataType("string", StringType)
   checkDataType("ChaR(5)", CharType(5))
   checkDataType("ChaRacter(5)", CharType(5))
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
index 37e2d9b..7b13fa9 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -465,6 +465,9 @@ class ExpressionParserSuite extends AnalysisTest {
 intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the 
TIMESTAMP value")
 
 // Timestamp without time zone
+assertEqual("tImEstAmp_Ntz '2016-03-11 20:54:00.000'",
+  Literal(LocalDateTime.parse("2016-03-11T20:54:00.000")))
+

[spark] branch branch-3.2 updated: [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ

2021-07-05 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 1ec37dd  [SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ
1ec37dd is described below

commit 1ec37dd164ae64c78bdccd8c0604b4013a692015
Author: Gengliang Wang 
AuthorDate: Mon Jul 5 22:30:44 2021 +0300

[SPARK-35977][SQL] Support non-reserved keyword TIMESTAMP_NTZ

### What changes were proposed in this pull request?

Support new keyword TIMESTAMP_NTZ, which can be used for:

- timestamp without time zone data type in DDL
- timestamp without time zone data type in Cast clause.
- timestamp without time zone data type literal

### Why are the changes needed?

Users can use `TIMESTAMP_NTZ` in DDL/Cast/Literals for the timestamp 
without time zone type directly.

### Does this PR introduce _any_ user-facing change?

No, the new timestamp type is not released yet.

### How was this patch tested?

Unit test

Closes #33221 from gengliangwang/timstamp_ntz.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 5f44acff3df51721fe891ea50c0a5bcf3a37a719)
Signed-off-by: Max Gekk 
---
 .../main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala  | 4 
 .../org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala| 1 +
 .../org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala  | 3 +++
 3 files changed, 8 insertions(+)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 5b9107f..c650cf0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2125,6 +2125,9 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   val zoneId = getZoneId(conf.sessionLocalTimeZone)
   val specialDate = convertSpecialDate(value, zoneId).map(Literal(_, 
DateType))
   specialDate.getOrElse(toLiteral(stringToDate, DateType))
+case "TIMESTAMP_NTZ" =>
+  val specialTs = convertSpecialTimestampNTZ(value).map(Literal(_, 
TimestampNTZType))
+  specialTs.getOrElse(toLiteral(stringToTimestampWithoutTimeZone, 
TimestampNTZType))
 case "TIMESTAMP" =>
   def constructTimestampLTZLiteral(value: String): Literal = {
 val zoneId = getZoneId(conf.sessionLocalTimeZone)
@@ -2525,6 +2528,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   case ("double", Nil) => DoubleType
   case ("date", Nil) => DateType
   case ("timestamp", Nil) => SQLConf.get.timestampType
+  case ("timestamp_ntz", Nil) => TimestampNTZType
   case ("string", Nil) => StringType
   case ("character" | "char", length :: Nil) => 
CharType(length.getText.toInt)
   case ("varchar", length :: Nil) => VarcharType(length.getText.toInt)
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
index a6b78e0..d34 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala
@@ -58,6 +58,7 @@ class DataTypeParserSuite extends SparkFunSuite with 
SQLHelper {
   checkDataType("deC", DecimalType.USER_DEFAULT)
   checkDataType("DATE", DateType)
   checkDataType("timestamp", TimestampType)
+  checkDataType("timestamp_ntz", TimestampNTZType)
   checkDataType("string", StringType)
   checkDataType("ChaR(5)", CharType(5))
   checkDataType("ChaRacter(5)", CharType(5))
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
index 37e2d9b..7b13fa9 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -465,6 +465,9 @@ class ExpressionParserSuite extends AnalysisTest {
 intercept("timestamP '2016-33-11 20:54:00.000'", "Cannot parse the 
TIMESTAMP value")
 
 // Timestamp without time zone
+assertEqual("tImEstAmp_Ntz &#

[spark] branch branch-3.2 updated: [SPARK-35983][SQL] Allow from_json/to_json for map types where value types are day-time intervals

2021-07-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 634b2e2  [SPARK-35983][SQL] Allow from_json/to_json for map types 
where value types are day-time intervals
634b2e2 is described below

commit 634b2e265c1a95af2f2df35f8ecb9d05d4fe752f
Author: Kousuke Saruta 
AuthorDate: Tue Jul 6 11:06:56 2021 +0300

[SPARK-35983][SQL] Allow from_json/to_json for map types where value types 
are day-time intervals

### What changes were proposed in this pull request?

This PR fixes two issues. One is that `to_json` doesn't support `map` types 
where value types are `day-time` interval types like:
```
spark-sql> select to_json(map('a', interval '1 2:3:4' day to second));
21/07/06 14:53:58 ERROR SparkSQLDriver: Failed in [select to_json(map('a', 
interval '1 2:3:4' day to second))]
java.lang.RuntimeException: Failed to convert value 9378400 (class of 
class java.lang.Long) with the type of DayTimeIntervalType(0,3) to JSON.
```
The other issue is that even if the issue of `to_json` is resolved, 
`from_json` doesn't support to convert `day-time` interval string to JSON. So 
the result of following query will be `null`.
```
spark-sql> select from_json(to_json(map('a', interval '1 2:3:4' day to 
second)), 'a interval day to second');
{"a":null}
```

### Why are the changes needed?

There should be no reason why day-time intervals cannot used as map value 
types.
`CalendarIntervalTypes` can do it.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33225 from sarutak/json-dtinterval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit c8ff613c3cd0d04ebfaf57feeb67e21b3c8410a2)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/json/JacksonGenerator.scala |  9 +
 .../spark/sql/catalyst/json/JacksonParser.scala|  7 
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 44 +-
 3 files changed, 59 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
index 9777d56..9bd0546 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala
@@ -156,6 +156,15 @@ private[sql] class JacksonGenerator(
   end)
 gen.writeString(ymString)
 
+case DayTimeIntervalType(start, end) =>
+  (row: SpecializedGetters, ordinal: Int) =>
+val dtString = IntervalUtils.toDayTimeIntervalString(
+  row.getLong(ordinal),
+  IntervalStringStyles.ANSI_STYLE,
+  start,
+  end)
+gen.writeString(dtString)
+
 case BinaryType =>
   (row: SpecializedGetters, ordinal: Int) =>
 gen.writeBinary(row.getBinary(ordinal))
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
index 2aa735d..8a1191c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala
@@ -302,6 +302,13 @@ class JacksonParser(
   Integer.valueOf(expr.eval(EmptyRow).asInstanceOf[Int])
   }
 
+case dt: DayTimeIntervalType => (parser: JsonParser) =>
+  parseJsonToken[java.lang.Long](parser, dataType) {
+case VALUE_STRING =>
+  val expr = Cast(Literal(parser.getText), dt)
+  java.lang.Long.valueOf(expr.eval(EmptyRow).asInstanceOf[Long])
+  }
+
 case st: StructType =>
   val fieldConverters = st.map(_.dataType).map(makeConverter).toArray
   (parser: JsonParser) => parseJsonToken[InternalRow](parser, dataType) {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
index c2bea8c..82cca2b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.sql
 
 import java.text.SimpleDateFormat
-import java.time.Period
+import java.time.{Duration, Period}
 import java.util.Locale
 
 import collection.JavaConverters._
@@ -28,6 +28,7 @@ import org.apache.spark.sql.functions._
 import org.apache.s

[spark] branch master updated (87282f0 -> c8ff613)

2021-07-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 87282f0  [SPARK-35972][SQL] When replace ExtractValue in 
NestedColumnAliasing we should use semanticEquals
 add c8ff613  [SPARK-35983][SQL] Allow from_json/to_json for map types 
where value types are day-time intervals

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/json/JacksonGenerator.scala |  9 +
 .../spark/sql/catalyst/json/JacksonParser.scala|  7 
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 44 +-
 3 files changed, 59 insertions(+), 1 deletion(-)

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



[spark] branch master updated: [SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code about parse string to DT/YM

2021-07-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 26d1bb1  [SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code 
about parse string to DT/YM
26d1bb1 is described below

commit 26d1bb16bc565dbcb1a3f536dc78cd87be6c2468
Author: Angerszh 
AuthorDate: Tue Jul 6 13:51:06 2021 +0300

[SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code about parse 
string to DT/YM

### What changes were proposed in this pull request?
 Refactor code about parse string to DT/YM intervals.

### Why are the changes needed?
Extracting the common code about parse string to DT/YM should improve code 
maintenance.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Existed UT.

Closes #33217 from AngersZh/SPARK-35735-35768.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 201 ++---
 .../sql/catalyst/expressions/CastSuiteBase.scala   |  28 ++-
 2 files changed, 123 insertions(+), 106 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 30a2fa5..b174165 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,7 +29,7 @@ import 
org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, 
YearMonthIntervalType => YM}
+import org.apache.spark.sql.types.{DataType, DayTimeIntervalType => DT, 
Decimal, YearMonthIntervalType => YM}
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 // The style of textual representation of intervals
@@ -110,7 +110,7 @@ object IntervalUtils {
   private val yearMonthIndividualLiteralRegex =
 
(s"(?i)^INTERVAL\\s+([+|-])?'$yearMonthIndividualPatternString'\\s+(YEAR|MONTH)$$").r
 
-  private def getSign(firstSign: String, secondSign: String): Int = {
+  private def finalSign(firstSign: String, secondSign: String = null): Int = {
 (firstSign, secondSign) match {
   case ("-", "-") => 1
   case ("-", _) => -1
@@ -119,6 +119,39 @@ object IntervalUtils {
 }
   }
 
+  private def throwIllegalIntervalFormatException(
+  input: UTF8String,
+  startFiled: Byte,
+  endField: Byte,
+  intervalStr: String,
+  typeName: String,
+  fallBackNotice: Option[String] = None) = {
+throw new IllegalArgumentException(
+  s"Interval string does not match $intervalStr format of " +
+s"${supportedFormat((startFiled, endField)).map(format => 
s"`$format`").mkString(", ")} " +
+s"when cast to $typeName: ${input.toString}" +
+s"${fallBackNotice.map(s => s", $s").getOrElse("")}")
+  }
+
+  private def checkIntervalStringDataType(
+  input: UTF8String,
+  targetStartField: Byte,
+  targetEndField: Byte,
+  inputIntervalType: DataType,
+  fallBackNotice: Option[String] = None): Unit = {
+val (intervalStr, typeName, inputStartField, inputEndField) = 
inputIntervalType match {
+  case DT(startField, endField) =>
+("day-time", DT(targetStartField, targetEndField).typeName, 
startField, endField)
+  case YM(startField, endField) =>
+("year-month", YM(targetStartField, targetEndField).typeName, 
startField, endField)
+}
+if (targetStartField != inputStartField || targetEndField != 
inputEndField) {
+  throwIllegalIntervalFormatException(
+input, targetStartField, targetEndField, intervalStr, typeName, 
fallBackNotice)
+}
+  }
+
+
   val supportedFormat = Map(
 (YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO 
MONTH"),
 (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
@@ -140,56 +173,41 @@ object IntervalUtils {
   startField: Byte,
   endField: Byte): Int = {
 
-def checkStringIntervalType(targetStartField: Byte, targetEndField: Byte): 
Unit = {
-  if (startField != targetStartField || endField != targetEndField) {
-throw new IllegalArgumentException(s"Interval string does not match 
year-month for

[spark] branch branch-3.2 updated: [SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code about parse string to DT/YM

2021-07-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new b53d285  [SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code 
about parse string to DT/YM
b53d285 is described below

commit b53d285f72a918abeafaf7517281d08cf57beb64
Author: Angerszh 
AuthorDate: Tue Jul 6 13:51:06 2021 +0300

[SPARK-36023][SPARK-35735][SPARK-35768][SQL] Refactor code about parse 
string to DT/YM

### What changes were proposed in this pull request?
 Refactor code about parse string to DT/YM intervals.

### Why are the changes needed?
Extracting the common code about parse string to DT/YM should improve code 
maintenance.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Existed UT.

Closes #33217 from AngersZh/SPARK-35735-35768.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
(cherry picked from commit 26d1bb16bc565dbcb1a3f536dc78cd87be6c2468)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 201 ++---
 .../sql/catalyst/expressions/CastSuiteBase.scala   |  28 ++-
 2 files changed, 123 insertions(+), 106 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 30a2fa5..b174165 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,7 +29,7 @@ import 
org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, 
YearMonthIntervalType => YM}
+import org.apache.spark.sql.types.{DataType, DayTimeIntervalType => DT, 
Decimal, YearMonthIntervalType => YM}
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 // The style of textual representation of intervals
@@ -110,7 +110,7 @@ object IntervalUtils {
   private val yearMonthIndividualLiteralRegex =
 
(s"(?i)^INTERVAL\\s+([+|-])?'$yearMonthIndividualPatternString'\\s+(YEAR|MONTH)$$").r
 
-  private def getSign(firstSign: String, secondSign: String): Int = {
+  private def finalSign(firstSign: String, secondSign: String = null): Int = {
 (firstSign, secondSign) match {
   case ("-", "-") => 1
   case ("-", _) => -1
@@ -119,6 +119,39 @@ object IntervalUtils {
 }
   }
 
+  private def throwIllegalIntervalFormatException(
+  input: UTF8String,
+  startFiled: Byte,
+  endField: Byte,
+  intervalStr: String,
+  typeName: String,
+  fallBackNotice: Option[String] = None) = {
+throw new IllegalArgumentException(
+  s"Interval string does not match $intervalStr format of " +
+s"${supportedFormat((startFiled, endField)).map(format => 
s"`$format`").mkString(", ")} " +
+s"when cast to $typeName: ${input.toString}" +
+s"${fallBackNotice.map(s => s", $s").getOrElse("")}")
+  }
+
+  private def checkIntervalStringDataType(
+  input: UTF8String,
+  targetStartField: Byte,
+  targetEndField: Byte,
+  inputIntervalType: DataType,
+  fallBackNotice: Option[String] = None): Unit = {
+val (intervalStr, typeName, inputStartField, inputEndField) = 
inputIntervalType match {
+  case DT(startField, endField) =>
+("day-time", DT(targetStartField, targetEndField).typeName, 
startField, endField)
+  case YM(startField, endField) =>
+("year-month", YM(targetStartField, targetEndField).typeName, 
startField, endField)
+}
+if (targetStartField != inputStartField || targetEndField != 
inputEndField) {
+  throwIllegalIntervalFormatException(
+input, targetStartField, targetEndField, intervalStr, typeName, 
fallBackNotice)
+}
+  }
+
+
   val supportedFormat = Map(
 (YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO 
MONTH"),
 (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),
@@ -140,56 +173,41 @@ object IntervalUtils {
   startField: Byte,
   endField: Byte): Int = {
 
-def checkStringIntervalType(targetStartField: Byte, targetEndField: Byte): 
Unit = {
-  if (startField != targetStartField || endFie

[spark] branch master updated: [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by hour to minute and unit case-sensitive issue

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 3953754  [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex 
can cover by hour to minute and unit case-sensitive issue
3953754 is described below

commit 3953754f36656e1a0bee16b89fae0142f172a91a
Author: Angerszh 
AuthorDate: Wed Jul 7 12:37:19 2021 +0300

[SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by 
hour to minute and unit case-sensitive issue

### What changes were proposed in this pull request?
When cast `10:10` to interval minute to second,  it can be catch by hour to 
minute regex, here to fix this.

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added UT

Closes #33242 from AngersZh/SPARK-35735-FOLLOWUP.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 92 +++---
 .../sql/catalyst/expressions/CastSuiteBase.scala   |  4 +
 2 files changed, 48 insertions(+), 48 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 b174165..ad87f2a 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.util
 
 import java.time.{Duration, Period}
 import java.time.temporal.ChronoUnit
+import java.util.Locale
 import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable
@@ -173,15 +174,14 @@ object IntervalUtils {
   startField: Byte,
   endField: Byte): Int = {
 
-def checkYMIntervalStringDataType(ym: YM): Unit =
-  checkIntervalStringDataType(input, startField, endField, ym)
+def checkTargetType(targetStartField: Byte, targetEndField: Byte): Boolean 
=
+  startField == targetStartField && endField == targetEndField
 
 input.trimAll().toString match {
-  case yearMonthRegex(sign, year, month) =>
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.MONTH))
+  case yearMonthRegex(sign, year, month) if checkTargetType(YM.YEAR, 
YM.MONTH) =>
 toYMInterval(year, month, finalSign(sign))
-  case yearMonthLiteralRegex(firstSign, secondSign, year, month) =>
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.MONTH))
+  case yearMonthLiteralRegex(firstSign, secondSign, year, month)
+if checkTargetType(YM.YEAR, YM.MONTH) =>
 toYMInterval(year, month, finalSign(firstSign, secondSign))
   case yearMonthIndividualRegex(firstSign, value) =>
 safeToInterval("year-month") {
@@ -195,15 +195,16 @@ object IntervalUtils {
   input, startField, endField, "year-month", YM(startField, 
endField).typeName)
   }
 }
-  case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, 
suffix) =>
+  case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, unit) 
=>
 safeToInterval("year-month") {
   val sign = finalSign(firstSign, secondSign)
-  if ("YEAR".equalsIgnoreCase(suffix)) {
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.YEAR))
-sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
-  } else {
-checkYMIntervalStringDataType(YM(YM.MONTH, YM.MONTH))
-Math.toIntExact(sign * value.toLong)
+  unit.toUpperCase(Locale.ROOT) match {
+case "YEAR" if checkTargetType(YM.YEAR, YM.YEAR) =>
+  sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
+case "MONTH" if checkTargetType(YM.MONTH, YM.MONTH) =>
+  Math.toIntExact(sign * value.toLong)
+case _ => throwIllegalIntervalFormatException(input, startField, 
endField,
+  "year-month", YM(startField, endField).typeName)
   }
 }
   case _ => throwIllegalIntervalFormatException(input, startField, 
endField,
@@ -303,48 +304,45 @@ object IntervalUtils {
   }
 }
 
-def checkDTIntervalStringDataType(dt: DT): Unit =
-  checkIntervalStringDataType(input, startField, endField, dt, 
Some(fallbackNotice))
+def checkTargetType(targetStartField: Byte, targetEndField: Byte): Boolean 
=
+  startField == targetStartField && endField == targetEndField
 
 input.trimAll().toString match {
-  case dayHourRegex(sign, day, hour) =>
-checkDTIntervalStringDataType(DT(DT.DAY, DT.HOUR))
+  case dayH

[spark] branch branch-3.2 updated: [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by hour to minute and unit case-sensitive issue

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new dd038aa  [SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex 
can cover by hour to minute and unit case-sensitive issue
dd038aa is described below

commit dd038aacd454e1a70ef2b2598fb504d8ad04b82a
Author: Angerszh 
AuthorDate: Wed Jul 7 12:37:19 2021 +0300

[SPARK-35735][SQL][FOLLOWUP] Fix case minute to second regex can cover by 
hour to minute and unit case-sensitive issue

### What changes were proposed in this pull request?
When cast `10:10` to interval minute to second,  it can be catch by hour to 
minute regex, here to fix this.

### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added UT

Closes #33242 from AngersZh/SPARK-35735-FOLLOWUP.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
(cherry picked from commit 3953754f36656e1a0bee16b89fae0142f172a91a)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 92 +++---
 .../sql/catalyst/expressions/CastSuiteBase.scala   |  4 +
 2 files changed, 48 insertions(+), 48 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 b174165..ad87f2a 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.util
 
 import java.time.{Duration, Period}
 import java.time.temporal.ChronoUnit
+import java.util.Locale
 import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable
@@ -173,15 +174,14 @@ object IntervalUtils {
   startField: Byte,
   endField: Byte): Int = {
 
-def checkYMIntervalStringDataType(ym: YM): Unit =
-  checkIntervalStringDataType(input, startField, endField, ym)
+def checkTargetType(targetStartField: Byte, targetEndField: Byte): Boolean 
=
+  startField == targetStartField && endField == targetEndField
 
 input.trimAll().toString match {
-  case yearMonthRegex(sign, year, month) =>
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.MONTH))
+  case yearMonthRegex(sign, year, month) if checkTargetType(YM.YEAR, 
YM.MONTH) =>
 toYMInterval(year, month, finalSign(sign))
-  case yearMonthLiteralRegex(firstSign, secondSign, year, month) =>
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.MONTH))
+  case yearMonthLiteralRegex(firstSign, secondSign, year, month)
+if checkTargetType(YM.YEAR, YM.MONTH) =>
 toYMInterval(year, month, finalSign(firstSign, secondSign))
   case yearMonthIndividualRegex(firstSign, value) =>
 safeToInterval("year-month") {
@@ -195,15 +195,16 @@ object IntervalUtils {
   input, startField, endField, "year-month", YM(startField, 
endField).typeName)
   }
 }
-  case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, 
suffix) =>
+  case yearMonthIndividualLiteralRegex(firstSign, secondSign, value, unit) 
=>
 safeToInterval("year-month") {
   val sign = finalSign(firstSign, secondSign)
-  if ("YEAR".equalsIgnoreCase(suffix)) {
-checkYMIntervalStringDataType(YM(YM.YEAR, YM.YEAR))
-sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
-  } else {
-checkYMIntervalStringDataType(YM(YM.MONTH, YM.MONTH))
-Math.toIntExact(sign * value.toLong)
+  unit.toUpperCase(Locale.ROOT) match {
+case "YEAR" if checkTargetType(YM.YEAR, YM.YEAR) =>
+  sign * Math.toIntExact(value.toLong * MONTHS_PER_YEAR)
+case "MONTH" if checkTargetType(YM.MONTH, YM.MONTH) =>
+  Math.toIntExact(sign * value.toLong)
+case _ => throwIllegalIntervalFormatException(input, startField, 
endField,
+  "year-month", YM(startField, endField).typeName)
   }
 }
   case _ => throwIllegalIntervalFormatException(input, startField, 
endField,
@@ -303,48 +304,45 @@ object IntervalUtils {
   }
 }
 
-def checkDTIntervalStringDataType(dt: DT): Unit =
-  checkIntervalStringDataType(input, startField, endField, dt, 
Some(fallbackNotice))
+def checkTargetType(targetStartField: Byte, targetEndField: Byte): Boolean 
=
+  startField == targetStartField && endField == targetEndField
 
 input.trimAll().toString match {
-  case day

[spark] branch master updated: [SPARK-36017][SQL] Support TimestampNTZType in expression ApproximatePercentile

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new cc4463e  [SPARK-36017][SQL] Support TimestampNTZType in expression 
ApproximatePercentile
cc4463e is described below

commit cc4463e818749faaf648ec71699d1e2fd3828c3f
Author: gengjiaan 
AuthorDate: Wed Jul 7 12:41:11 2021 +0300

[SPARK-36017][SQL] Support TimestampNTZType in expression 
ApproximatePercentile

### What changes were proposed in this pull request?
The current `ApproximatePercentile` supports `TimestampType`, but not 
supports timestamp without time zone yet.
This PR will add the function.

### Why are the changes needed?
`ApproximatePercentile` need supports `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'. `ApproximatePercentile` accepts `TimestampNTZType`.

### How was this patch tested?
New tests.

Closes #33241 from beliefer/SPARK-36017.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
---
 .../expressions/aggregate/ApproximatePercentile.scala  | 10 +-
 .../apache/spark/sql/ApproximatePercentileQuerySuite.scala | 14 +-
 2 files changed, 14 insertions(+), 10 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 78e64bf..8cce79c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -92,9 +92,9 @@ case class ApproximatePercentile(
   private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
-// Support NumericType, DateType and TimestampType since their internal 
types are all numeric,
-// and can be easily cast to double for processing.
-Seq(TypeCollection(NumericType, DateType, TimestampType),
+// Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
+// are all numeric, and can be easily cast to double for processing.
+Seq(TypeCollection(NumericType, DateType, TimestampType, TimestampNTZType),
   TypeCollection(DoubleType, ArrayType(DoubleType, containsNull = false)), 
IntegralType)
   }
 
@@ -139,7 +139,7 @@ case class ApproximatePercentile(
   // Convert the value to a double value
   val doubleValue = child.dataType match {
 case DateType => value.asInstanceOf[Int].toDouble
-case TimestampType => value.asInstanceOf[Long].toDouble
+case TimestampType | TimestampNTZType => 
value.asInstanceOf[Long].toDouble
 case n: NumericType => 
n.numeric.toDouble(value.asInstanceOf[n.InternalType])
 case other: DataType =>
   throw QueryExecutionErrors.dataTypeUnexpectedError(other)
@@ -158,7 +158,7 @@ case class ApproximatePercentile(
 val doubleResult = buffer.getPercentiles(percentages)
 val result = child.dataType match {
   case DateType => doubleResult.map(_.toInt)
-  case TimestampType => doubleResult.map(_.toLong)
+  case TimestampType | TimestampNTZType => doubleResult.map(_.toLong)
   case ByteType => doubleResult.map(_.toByte)
   case ShortType => doubleResult.map(_.toShort)
   case IntegerType => doubleResult.map(_.toInt)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 4991e39..5ff15c9 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.sql.{Date, Timestamp}
+import java.time.LocalDateTime
 
 import 
org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
 import 
org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY
@@ -89,23 +90,26 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   test("percentile_approx, different column types") {
 withTempView(table) {
   val intSeq = 1 to 1000
-  val data: Seq[(java.math.BigDecimal, Date, Timestamp)] = intSeq.map { i 
=>
-(new java.math.BigDecimal(i), DateTimeUtils.toJavaDate(i), 
DateTimeUtils.toJavaTimestamp(i))
+  val data: Seq[(java.math.BigDecimal, Date, Timestamp, LocalDateTime)] = 
intSeq.map { i =>
+ 

[spark] branch branch-3.2 updated: [SPARK-36017][SQL] Support TimestampNTZType in expression ApproximatePercentile

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 25ea296  [SPARK-36017][SQL] Support TimestampNTZType in expression 
ApproximatePercentile
25ea296 is described below

commit 25ea296c3c4295d78a573d98edbdd8f1de0e0447
Author: gengjiaan 
AuthorDate: Wed Jul 7 12:41:11 2021 +0300

[SPARK-36017][SQL] Support TimestampNTZType in expression 
ApproximatePercentile

### What changes were proposed in this pull request?
The current `ApproximatePercentile` supports `TimestampType`, but not 
supports timestamp without time zone yet.
This PR will add the function.

### Why are the changes needed?
`ApproximatePercentile` need supports `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'. `ApproximatePercentile` accepts `TimestampNTZType`.

### How was this patch tested?
New tests.

Closes #33241 from beliefer/SPARK-36017.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
(cherry picked from commit cc4463e818749faaf648ec71699d1e2fd3828c3f)
Signed-off-by: Max Gekk 
---
 .../expressions/aggregate/ApproximatePercentile.scala  | 10 +-
 .../apache/spark/sql/ApproximatePercentileQuerySuite.scala | 14 +-
 2 files changed, 14 insertions(+), 10 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
index 78e64bf..8cce79c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala
@@ -92,9 +92,9 @@ case class ApproximatePercentile(
   private lazy val accuracy: Long = 
accuracyExpression.eval().asInstanceOf[Number].longValue
 
   override def inputTypes: Seq[AbstractDataType] = {
-// Support NumericType, DateType and TimestampType since their internal 
types are all numeric,
-// and can be easily cast to double for processing.
-Seq(TypeCollection(NumericType, DateType, TimestampType),
+// Support NumericType, DateType, TimestampType and TimestampNTZType since 
their internal types
+// are all numeric, and can be easily cast to double for processing.
+Seq(TypeCollection(NumericType, DateType, TimestampType, TimestampNTZType),
   TypeCollection(DoubleType, ArrayType(DoubleType, containsNull = false)), 
IntegralType)
   }
 
@@ -139,7 +139,7 @@ case class ApproximatePercentile(
   // Convert the value to a double value
   val doubleValue = child.dataType match {
 case DateType => value.asInstanceOf[Int].toDouble
-case TimestampType => value.asInstanceOf[Long].toDouble
+case TimestampType | TimestampNTZType => 
value.asInstanceOf[Long].toDouble
 case n: NumericType => 
n.numeric.toDouble(value.asInstanceOf[n.InternalType])
 case other: DataType =>
   throw QueryExecutionErrors.dataTypeUnexpectedError(other)
@@ -158,7 +158,7 @@ case class ApproximatePercentile(
 val doubleResult = buffer.getPercentiles(percentages)
 val result = child.dataType match {
   case DateType => doubleResult.map(_.toInt)
-  case TimestampType => doubleResult.map(_.toLong)
+  case TimestampType | TimestampNTZType => doubleResult.map(_.toLong)
   case ByteType => doubleResult.map(_.toByte)
   case ShortType => doubleResult.map(_.toShort)
   case IntegerType => doubleResult.map(_.toInt)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 4991e39..5ff15c9 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.sql.{Date, Timestamp}
+import java.time.LocalDateTime
 
 import 
org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
 import 
org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY
@@ -89,23 +90,26 @@ class ApproximatePercentileQuerySuite extends QueryTest 
with SharedSparkSession
   test("percentile_approx, different column types") {
 withTempView(table) {
   val intSeq = 1 to 1000
-  val data: Seq[(java.math.BigDecimal, Date, Timestamp)] = intSeq.map { i 
=>
-(new java.math.BigDecimal(i), DateTimeUtils.toJavaDate(i), 
DateTimeUtils.toJavaTimestamp(i))
+  val data: S

[spark] branch master updated (55373b1 -> be382a6)

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 55373b1  [SPARK-35907][CORE] Instead of File#mkdirs, 
Files#createDirectories is expected
 add be382a6  [SPARK-36016][SQL] Support TimestampNTZType in expression 
ApproxCountDistinctForIntervals

No new revisions were added by this update.

Summary of changes:
 .../expressions/aggregate/ApproxCountDistinctForIntervals.scala   | 6 +++---
 .../aggregate/ApproxCountDistinctForIntervalsSuite.scala  | 8 ++--
 2 files changed, 9 insertions(+), 5 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36016][SQL] Support TimestampNTZType in expression ApproxCountDistinctForIntervals

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 0c7972b  [SPARK-36016][SQL] Support TimestampNTZType in expression 
ApproxCountDistinctForIntervals
0c7972b is described below

commit 0c7972ba5f9c4ab991b8ade00df409fb4392788a
Author: gengjiaan 
AuthorDate: Wed Jul 7 20:22:46 2021 +0300

[SPARK-36016][SQL] Support TimestampNTZType in expression 
ApproxCountDistinctForIntervals

### What changes were proposed in this pull request?
The current `ApproxCountDistinctForInterval`s supports `TimestampType`, but 
not supports timestamp without time zone yet.
This PR will add the function.

### Why are the changes needed?
`ApproxCountDistinctForInterval` need supports `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'. `ApproxCountDistinctForInterval` accepts `TimestampNTZType`.

### How was this patch tested?
New tests.

Closes #33243 from beliefer/SPARK-36016.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
(cherry picked from commit be382a6285fcbf374faeec1298371952a7bf1193)
Signed-off-by: Max Gekk 
---
 .../expressions/aggregate/ApproxCountDistinctForIntervals.scala   | 6 +++---
 .../aggregate/ApproxCountDistinctForIntervalsSuite.scala  | 8 ++--
 2 files changed, 9 insertions(+), 5 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala
index 19e212d..a7e9a22 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervals.scala
@@ -61,7 +61,7 @@ case class ApproxCountDistinctForIntervals(
   }
 
   override def inputTypes: Seq[AbstractDataType] = {
-Seq(TypeCollection(NumericType, TimestampType, DateType), ArrayType)
+Seq(TypeCollection(NumericType, TimestampType, DateType, 
TimestampNTZType), ArrayType)
   }
 
   // Mark as lazy so that endpointsExpression is not evaluated during tree 
transformation.
@@ -79,7 +79,7 @@ case class ApproxCountDistinctForIntervals(
   TypeCheckFailure("The endpoints provided must be constant literals")
 } else {
   endpointsExpression.dataType match {
-case ArrayType(_: NumericType | DateType | TimestampType, _) =>
+case ArrayType(_: NumericType | DateType | TimestampType | 
TimestampNTZType, _) =>
   if (endpoints.length < 2) {
 TypeCheckFailure("The number of endpoints must be >= 2 to 
construct intervals")
   } else {
@@ -122,7 +122,7 @@ case class ApproxCountDistinctForIntervals(
   n.numeric.toDouble(value.asInstanceOf[n.InternalType])
 case _: DateType =>
   value.asInstanceOf[Int].toDouble
-case _: TimestampType =>
+case TimestampType | TimestampNTZType =>
   value.asInstanceOf[Long].toDouble
   }
 
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervalsSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervalsSuite.scala
index 73f18d4..9d53673 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervalsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproxCountDistinctForIntervalsSuite.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.expressions.aggregate
 
 import java.sql.{Date, Timestamp}
+import java.time.LocalDateTime
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.catalyst.InternalRow
@@ -38,7 +39,7 @@ class ApproxCountDistinctForIntervalsSuite extends 
SparkFunSuite {
   assert(
 wrongColumn.checkInputDataTypes() match {
   case TypeCheckFailure(msg)
-if msg.contains("requires (numeric or timestamp or date) type") => 
true
+if msg.contains("requires (numeric or timestamp or date or 
timestamp_ntz) type") => true
   case _ => false
 })
 }
@@ -199,7 +200,9 @@ class ApproxCountDistinctForIntervalsSuite extends 
SparkFunSuite {
   (intRecords.map(DateTimeUtils.toJavaDate),
   intEndpoints.map(DateTimeUtils.toJavaDate), DateType),
   (intRecords.map(DateTimeUtils.toJavaTimestamp(_)),
-  intEndpoints.map(DateTimeUtils.toJavaTimestamp(_)), TimestampType)
+  intEndpoin

[spark] branch master updated: [SPARK-36015][SQL] Support TimestampNTZType in the Window spec definition

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 62ff2ad  [SPARK-36015][SQL] Support TimestampNTZType in the Window 
spec definition
62ff2ad is described below

commit 62ff2add9444fbd54802548b3daf7cde5820feef
Author: gengjiaan 
AuthorDate: Wed Jul 7 20:27:05 2021 +0300

[SPARK-36015][SQL] Support TimestampNTZType in the Window spec definition

### What changes were proposed in this pull request?
The method `WindowSpecDefinition.isValidFrameType` doesn't consider 
`TimestampNTZType`. We should support it as for `TimestampType`.

### Why are the changes needed?
Support `TimestampNTZType` in the Window spec definition.

### Does this PR introduce _any_ user-facing change?
'Yes'. This PR allows users use  `TimestampNTZType` as the sort spec in 
window spec definition.

### How was this patch tested?
New tests.

Closes #33246 from beliefer/SPARK-36015.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/windowExpressions.scala   |  6 +--
 .../sql/execution/window/WindowExecBase.scala  | 10 ++--
 .../src/test/resources/sql-tests/inputs/window.sql |  9 
 .../resources/sql-tests/results/window.sql.out | 56 +-
 4 files changed, 73 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index 2555c6a..fc2e449 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -102,9 +102,9 @@ case class WindowSpecDefinition(
   private def isValidFrameType(ft: DataType): Boolean = 
(orderSpec.head.dataType, ft) match {
 case (DateType, IntegerType) => true
 case (DateType, _: YearMonthIntervalType) => true
-case (TimestampType, CalendarIntervalType) => true
-case (TimestampType, _: YearMonthIntervalType) => true
-case (TimestampType, _: DayTimeIntervalType) => true
+case (TimestampType | TimestampNTZType, CalendarIntervalType) => true
+case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) => true
+case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) => true
 case (a, b) => a == b
   }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
index 2aa0b02..f3b3b34 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.UnaryExecNode
-import org.apache.spark.sql.types.{CalendarIntervalType, DateType, 
DayTimeIntervalType, IntegerType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.sql.types._
 
 trait WindowExecBase extends UnaryExecNode {
   def windowExpression: Seq[NamedExpression]
@@ -96,10 +96,12 @@ trait WindowExecBase extends UnaryExecNode {
 val boundExpr = (expr.dataType, boundOffset.dataType) match {
   case (DateType, IntegerType) => DateAdd(expr, boundOffset)
   case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(expr, 
boundOffset)
-  case (TimestampType, CalendarIntervalType) => TimeAdd(expr, 
boundOffset, Some(timeZone))
-  case (TimestampType, _: YearMonthIntervalType) =>
+  case (TimestampType | TimestampNTZType, CalendarIntervalType) =>
+TimeAdd(expr, boundOffset, Some(timeZone))
+  case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
 TimestampAddYMInterval(expr, boundOffset, Some(timeZone))
-  case (TimestampType, _: DayTimeIntervalType) => TimeAdd(expr, 
boundOffset, Some(timeZone))
+  case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) =>
+TimeAdd(expr, boundOffset, Some(timeZone))
   case (a, b) if a == b => Add(expr, boundOffset)
 }
 val bound = MutableProjection.create(boundExpr :: Nil, child.output)
diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql 
b/sql/core/src/test/resources/sql-tests/inputs/window.sql
index 46d3629..9766aaf 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/window.sql
+++ b/sql/core/src/test/re

[spark] branch branch-3.2 updated: [SPARK-36015][SQL] Support TimestampNTZType in the Window spec definition

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 2fc57bba [SPARK-36015][SQL] Support TimestampNTZType in the Window 
spec definition
2fc57bba is described below

commit 2fc57bba31a9bea3b97582751a693acd268158e9
Author: gengjiaan 
AuthorDate: Wed Jul 7 20:27:05 2021 +0300

[SPARK-36015][SQL] Support TimestampNTZType in the Window spec definition

### What changes were proposed in this pull request?
The method `WindowSpecDefinition.isValidFrameType` doesn't consider 
`TimestampNTZType`. We should support it as for `TimestampType`.

### Why are the changes needed?
Support `TimestampNTZType` in the Window spec definition.

### Does this PR introduce _any_ user-facing change?
'Yes'. This PR allows users use  `TimestampNTZType` as the sort spec in 
window spec definition.

### How was this patch tested?
New tests.

Closes #33246 from beliefer/SPARK-36015.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
(cherry picked from commit 62ff2add9444fbd54802548b3daf7cde5820feef)
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/windowExpressions.scala   |  6 +--
 .../sql/execution/window/WindowExecBase.scala  | 10 ++--
 .../src/test/resources/sql-tests/inputs/window.sql |  9 
 .../resources/sql-tests/results/window.sql.out | 56 +-
 4 files changed, 73 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
index 2555c6a..fc2e449 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala
@@ -102,9 +102,9 @@ case class WindowSpecDefinition(
   private def isValidFrameType(ft: DataType): Boolean = 
(orderSpec.head.dataType, ft) match {
 case (DateType, IntegerType) => true
 case (DateType, _: YearMonthIntervalType) => true
-case (TimestampType, CalendarIntervalType) => true
-case (TimestampType, _: YearMonthIntervalType) => true
-case (TimestampType, _: DayTimeIntervalType) => true
+case (TimestampType | TimestampNTZType, CalendarIntervalType) => true
+case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) => true
+case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) => true
 case (a, b) => a == b
   }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
index 2aa0b02..f3b3b34 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.UnaryExecNode
-import org.apache.spark.sql.types.{CalendarIntervalType, DateType, 
DayTimeIntervalType, IntegerType, TimestampType, YearMonthIntervalType}
+import org.apache.spark.sql.types._
 
 trait WindowExecBase extends UnaryExecNode {
   def windowExpression: Seq[NamedExpression]
@@ -96,10 +96,12 @@ trait WindowExecBase extends UnaryExecNode {
 val boundExpr = (expr.dataType, boundOffset.dataType) match {
   case (DateType, IntegerType) => DateAdd(expr, boundOffset)
   case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(expr, 
boundOffset)
-  case (TimestampType, CalendarIntervalType) => TimeAdd(expr, 
boundOffset, Some(timeZone))
-  case (TimestampType, _: YearMonthIntervalType) =>
+  case (TimestampType | TimestampNTZType, CalendarIntervalType) =>
+TimeAdd(expr, boundOffset, Some(timeZone))
+  case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) =>
 TimestampAddYMInterval(expr, boundOffset, Some(timeZone))
-  case (TimestampType, _: DayTimeIntervalType) => TimeAdd(expr, 
boundOffset, Some(timeZone))
+  case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) =>
+TimeAdd(expr, boundOffset, Some(timeZone))
   case (a, b) if a == b => Add(expr, boundOffset)
 }
 val bound = MutableProjection.create(boundExpr :: Nil, child.output)
diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql 
b/sql/core/src/test/resources/sql-tests/inputs/window.sql
index 46d36

[spark] branch master updated (62ff2ad -> ea3333a)

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 62ff2ad  [SPARK-36015][SQL] Support TimestampNTZType in the Window 
spec definition
 add eaa  [SPARK-36021][SQL] Parse interval literals should support 
more than 2 digits

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/util/IntervalUtils.scala| 144 +--
 .../sql/catalyst/util/IntervalUtilsSuite.scala |  25 +--
 .../test/resources/sql-tests/inputs/interval.sql   |  21 +++
 .../sql-tests/results/ansi/interval.sql.out| 198 -
 .../resources/sql-tests/results/interval.sql.out   | 198 -
 .../sql-tests/results/postgreSQL/interval.sql.out  |  20 +--
 6 files changed, 467 insertions(+), 139 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36021][SQL] Parse interval literals should support more than 2 digits

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 74bfbcd  [SPARK-36021][SQL] Parse interval literals should support 
more than 2 digits
74bfbcd is described below

commit 74bfbcd6430b1a5d274c660224851ce1562813e4
Author: Angerszh 
AuthorDate: Wed Jul 7 20:31:29 2021 +0300

[SPARK-36021][SQL] Parse interval literals should support more than 2 digits

### What changes were proposed in this pull request?
For case
```
spark-sql> select interval '123456:12' minute to second;
Error in query:
requirement failed: Interval string must match day-time format of 
'^(?[+|-])?(?\d{1,2}):(?(\d{1,2})(\.(\d{1,9}))?)$': 
123456:12, set spark.sql.legacy.fromDayTimeString.enabled to true to restore 
the behavior before Spark 3.0.(line 1, pos 16)

== SQL ==
select interval '123456:12' minute to second
^^^
```

we should support hour/minute/second when for more than 2 digits when parse 
interval literal string

### Why are the changes needed?
Keep consistence

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added UT

Closes #33231 from AngersZh/SPARK-36021.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
(cherry picked from commit eaa200e586043e29e2f3566b95b6943b811f)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 144 +--
 .../sql/catalyst/util/IntervalUtilsSuite.scala |  25 +--
 .../test/resources/sql-tests/inputs/interval.sql   |  21 +++
 .../sql-tests/results/ansi/interval.sql.out| 198 -
 .../resources/sql-tests/results/interval.sql.out   | 198 -
 .../sql-tests/results/postgreSQL/interval.sql.out  |  20 +--
 6 files changed, 467 insertions(+), 139 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 ad87f2a..24bcad8 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
@@ -57,6 +57,12 @@ object IntervalUtils {
   }
   import IntervalUnit._
 
+  private val MAX_DAY = Long.MaxValue / MICROS_PER_DAY
+  private val MAX_HOUR = Long.MaxValue / MICROS_PER_HOUR
+  private val MAX_MINUTE = Long.MaxValue / MICROS_PER_MINUTE
+  private val MAX_SECOND = Long.MaxValue / MICROS_PER_SECOND
+  private val MIN_SECOND = Long.MinValue / MICROS_PER_SECOND
+
   def getYears(months: Int): Int = months / MONTHS_PER_YEAR
 
   def getYears(interval: CalendarInterval): Int = getYears(interval.months)
@@ -213,19 +219,25 @@ object IntervalUtils {
   }
 
   /**
-   * Parse YearMonth string in form: [+|-]-MM
+   * Parse year-month interval in form: [+|-]-MM
*
* adapted from HiveIntervalYearMonth.valueOf
*/
   def fromYearMonthString(input: String): CalendarInterval = {
+fromYearMonthString(input, YM.YEAR, YM.MONTH)
+  }
+
+  /**
+   * Parse year-month interval in form: [+|-]-MM
+   *
+   * adapted from HiveIntervalYearMonth.valueOf
+   * Below interval conversion patterns are supported:
+   * - YEAR TO (YEAR|MONTH)
+   */
+  def fromYearMonthString(input: String, startField: Byte, endField: Byte): 
CalendarInterval = {
 require(input != null, "Interval year-month string must be not null")
-input.trim match {
-  case yearMonthRegex(sign, yearStr, monthStr) =>
-new CalendarInterval(toYMInterval(yearStr, monthStr, finalSign(sign)), 
0, 0)
-  case _ =>
-throw new IllegalArgumentException(
-  s"Interval string does not match year-month format of 'y-m': $input")
-}
+val months = castStringToYMInterval(UTF8String.fromString(input), 
startField, endField)
+new CalendarInterval(months, 0, 0)
   }
 
   private def safeToInterval[T](interval: String)(f: => T): T = {
@@ -397,7 +409,7 @@ object IntervalUtils {
   secondStr: String,
   sign: Int): Long = {
 var micros = 0L
-val days = toLongWithRange(DAY, dayStr, 0, Int.MaxValue).toInt
+val days = toLongWithRange(DAY, dayStr, 0, MAX_DAY).toInt
 micros = Math.addExact(micros, sign * days * MICROS_PER_DAY)
 val hours = toLongWithRange(HOUR, hourStr, 0, 23)
 micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
@@ -413,7 +425,7 @@ object IntervalUtils {
   secondStr: String,
   sign: Int): Long = {
 var micros = 0L
-val hours = toLongWithRange(HOUR, hourStr, 0, 2562047788L)
+val hours = toLongWithRange(HOUR, hourStr, 0, MAX_HOUR)

[spark] branch master updated (23943e5 -> 39002cb)

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 23943e5  [SPARK-32577][SQL][TEST][FOLLOWUP] Fix the config value of 
shuffled hash join for all other test queries
 add 39002cb  [SPARK-36022][SQL] Respect interval fields in extract

No new revisions were added by this update.

Summary of changes:
 .../catalyst/expressions/intervalExpressions.scala | 24 ++--
 .../apache/spark/sql/IntervalFunctionsSuite.scala  | 64 ++
 2 files changed, 82 insertions(+), 6 deletions(-)
 create mode 100644 
sql/core/src/test/scala/org/apache/spark/sql/IntervalFunctionsSuite.scala

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



[spark] branch branch-3.2 updated: [SPARK-36022][SQL] Respect interval fields in extract

2021-07-07 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 429d178  [SPARK-36022][SQL] Respect interval fields in extract
429d178 is described below

commit 429d1780b3ab3267a5e22a857cf51458713bc208
Author: Kousuke Saruta 
AuthorDate: Thu Jul 8 09:40:57 2021 +0300

[SPARK-36022][SQL] Respect interval fields in extract

### What changes were proposed in this pull request?

This PR fixes an issue about `extract`.
`Extract` should process only existing fields of interval types. For 
example:

```
spark-sql> SELECT EXTRACT(MONTH FROM INTERVAL '2021-11' YEAR TO MONTH);
11
spark-sql> SELECT EXTRACT(MONTH FROM INTERVAL '2021' YEAR);
0
```
The last command should fail as the month field doesn't present in INTERVAL 
YEAR.

### Why are the changes needed?

Bug fix.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33247 from sarutak/fix-extract-interval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit 39002cb99514010f6d6cc2e575b9eab1694f04ef)
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/intervalExpressions.scala | 24 ++--
 .../apache/spark/sql/IntervalFunctionsSuite.scala  | 64 ++
 2 files changed, 82 insertions(+), 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 5d49007..5b111d1 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
@@ -29,6 +29,8 @@ import org.apache.spark.sql.catalyst.util.IntervalUtils._
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, MINUTE, 
SECOND}
+import org.apache.spark.sql.types.YearMonthIntervalType.{MONTH, YEAR}
 import org.apache.spark.unsafe.types.CalendarInterval
 
 abstract class ExtractIntervalPart[T](
@@ -125,33 +127,43 @@ object ExtractIntervalPart {
   source: Expression,
   errorHandleFunc: => Nothing): Expression = {
 (extractField.toUpperCase(Locale.ROOT), source.dataType) match {
-  case ("YEAR" | "Y" | "YEARS" | "YR" | "YRS", _: YearMonthIntervalType) =>
+  case ("YEAR" | "Y" | "YEARS" | "YR" | "YRS", 
YearMonthIntervalType(start, end))
+if isUnitInIntervalRange(YEAR, start, end) =>
 ExtractANSIIntervalYears(source)
   case ("YEAR" | "Y" | "YEARS" | "YR" | "YRS", CalendarIntervalType) =>
 ExtractIntervalYears(source)
-  case ("MONTH" | "MON" | "MONS" | "MONTHS", _: YearMonthIntervalType) =>
+  case ("MONTH" | "MON" | "MONS" | "MONTHS", YearMonthIntervalType(start, 
end))
+if isUnitInIntervalRange(MONTH, start, end) =>
 ExtractANSIIntervalMonths(source)
   case ("MONTH" | "MON" | "MONS" | "MONTHS", CalendarIntervalType) =>
 ExtractIntervalMonths(source)
-  case ("DAY" | "D" | "DAYS", _: DayTimeIntervalType) =>
+  case ("DAY" | "D" | "DAYS", DayTimeIntervalType(start, end))
+if isUnitInIntervalRange(DAY, start, end) =>
 ExtractANSIIntervalDays(source)
   case ("DAY" | "D" | "DAYS", CalendarIntervalType) =>
 ExtractIntervalDays(source)
-  case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", _: DayTimeIntervalType) =>
+  case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", DayTimeIntervalType(start, 
end))
+if isUnitInIntervalRange(HOUR, start, end) =>
 ExtractANSIIntervalHours(source)
   case ("HOUR" | "H" | "HOURS" | "HR" | "HRS", CalendarIntervalType) =>
 ExtractIntervalHours(source)
-  case ("MINUTE" | "M" | "MIN" | "MINS" | "MINUTES", _: 
DayTimeIntervalType) =>
+  case ("MINUTE" | "M" | "MIN" | "MINS" | "MINUT

[spark] branch master updated (39002cb -> 89aa16b)

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 39002cb  [SPARK-36022][SQL] Respect interval fields in extract
 add 89aa16b  [SPARK-36021][SQL][FOLLOWUP] DT/YM func use field byte to 
keep consistence

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/parser/AstBuilder.scala | 17 ---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 24 ++
 .../catalyst/parser/ExpressionParserSuite.scala|  3 ++-
 .../sql/catalyst/util/IntervalUtilsSuite.scala | 13 
 4 files changed, 21 insertions(+), 36 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36021][SQL][FOLLOWUP] DT/YM func use field byte to keep consistence

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 2776e8a  [SPARK-36021][SQL][FOLLOWUP] DT/YM func use field byte to 
keep consistence
2776e8a is described below

commit 2776e8aa4792a7b2e95d5bbbd76cea0f9554503c
Author: Angerszh 
AuthorDate: Thu Jul 8 12:22:04 2021 +0300

[SPARK-36021][SQL][FOLLOWUP] DT/YM func use field byte to keep consistence

### What changes were proposed in this pull request?
With more thought, all DT/YM function use field byte to keep consistence is 
better

### Why are the changes needed?
Keep code consistence

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Not need

Closes #33252 from AngersZh/SPARK-36021-FOLLOWUP.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
(cherry picked from commit 89aa16b4a838246cfb7bdc9318461485016f6252)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/parser/AstBuilder.scala | 17 ---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 24 ++
 .../catalyst/parser/ExpressionParserSuite.scala|  3 ++-
 .../sql/catalyst/util/IntervalUtilsSuite.scala | 13 
 4 files changed, 21 insertions(+), 36 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index d6363b5..4f1e53f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -40,7 +40,6 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.trees.CurrentOrigin
 import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils, 
IntervalUtils}
 import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, 
convertSpecialTimestamp, convertSpecialTimestampNTZ, getZoneId, stringToDate, 
stringToTimestamp, stringToTimestampWithoutTimeZone}
-import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit
 import org.apache.spark.sql.connector.catalog.{SupportsNamespaces, 
TableCatalog}
 import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
 import org.apache.spark.sql.connector.expressions.{ApplyTransform, 
BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, 
HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, 
YearsTransform}
@@ -2487,18 +2486,10 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] 
with SQLConfHelper with Logg
 (from, to) match {
   case ("year", "month") =>
 IntervalUtils.fromYearMonthString(value)
-  case ("day", "hour") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.DAY, 
IntervalUnit.HOUR)
-  case ("day", "minute") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.DAY, 
IntervalUnit.MINUTE)
-  case ("day", "second") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.DAY, 
IntervalUnit.SECOND)
-  case ("hour", "minute") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.HOUR, 
IntervalUnit.MINUTE)
-  case ("hour", "second") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.HOUR, 
IntervalUnit.SECOND)
-  case ("minute", "second") =>
-IntervalUtils.fromDayTimeString(value, IntervalUnit.MINUTE, 
IntervalUnit.SECOND)
+  case ("day", "hour") | ("day", "minute") | ("day", "second") | 
("hour", "minute") |
+   ("hour", "second") | ("minute", "second") =>
+IntervalUtils.fromDayTimeString(value,
+  DayTimeIntervalType.stringToField(from), 
DayTimeIntervalType.stringToField(to))
   case _ =>
 throw QueryParsingErrors.fromToIntervalUnsupportedError(from, to, 
ctx)
 }
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 24bcad8..7579a28 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
@@ -458,7 +458,7 @@ object IntervalUtils {
* adapted from HiveIntervalDayTime.valueOf
*/
   def fromDayTimeS

[spark] branch master updated (e071721 -> ee945e9)

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from e071721  [SPARK-36012][SQL] Add null flag in SHOW CREATE TABLE
 add ee945e9  [SPARK-36055][SQL] Assign pretty SQL string to TimestampNTZ 
literals

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/expressions/literals.scala  |  6 +-
 .../catalyst/expressions/LiteralExpressionSuite.scala  |  9 +
 .../sql-tests/results/timestampNTZ/datetime.sql.out| 18 +-
 3 files changed, 23 insertions(+), 10 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36055][SQL] Assign pretty SQL string to TimestampNTZ literals

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 9103c1f  [SPARK-36055][SQL] Assign pretty SQL string to TimestampNTZ 
literals
9103c1f is described below

commit 9103c1fe2332a60424077ca9ecffb24afa440c55
Author: Gengliang Wang 
AuthorDate: Thu Jul 8 21:42:50 2021 +0300

[SPARK-36055][SQL] Assign pretty SQL string to TimestampNTZ literals

### What changes were proposed in this pull request?

Currently the TimestampNTZ literals shows only long value instead of 
timestamp string in its SQL string and toString result.
Before changes (with default timestamp type as TIMESTAMP_NTZ)
```
– !query
select timestamp '2019-01-01\t'
– !query schema
struct<15463008:timestamp_ntz>
```

After changes:
```
– !query
select timestamp '2019-01-01\t'
– !query schema
struct
```
### Why are the changes needed?

Make the schema of TimestampNTZ literals readable.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test

Closes #33269 from gengliangwang/ntzLiteralString.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit ee945e99cc1d3979a2c24077a9ae786ce50bbe81)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/expressions/literals.scala  |  6 +-
 .../catalyst/expressions/LiteralExpressionSuite.scala  |  9 +
 .../sql-tests/results/timestampNTZ/datetime.sql.out| 18 +-
 3 files changed, 23 insertions(+), 10 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index a2270eb..ee40909 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -28,7 +28,7 @@ import java.lang.{Short => JavaShort}
 import java.math.{BigDecimal => JavaBigDecimal}
 import java.nio.charset.StandardCharsets
 import java.sql.{Date, Timestamp}
-import java.time.{Duration, Instant, LocalDate, LocalDateTime, Period}
+import java.time.{Duration, Instant, LocalDate, LocalDateTime, Period, 
ZoneOffset}
 import java.util
 import java.util.Objects
 import javax.xml.bind.DatatypeConverter
@@ -352,6 +352,8 @@ case class Literal (value: Any, dataType: DataType) extends 
LeafExpression {
   DateFormatter().format(value.asInstanceOf[Int])
 case TimestampType =>
   
TimestampFormatter.getFractionFormatter(timeZoneId).format(value.asInstanceOf[Long])
+case TimestampNTZType =>
+  
TimestampFormatter.getFractionFormatter(ZoneOffset.UTC).format(value.asInstanceOf[Long])
 case DayTimeIntervalType(startField, endField) =>
   toDayTimeIntervalString(value.asInstanceOf[Long], ANSI_STYLE, 
startField, endField)
 case YearMonthIntervalType(startField, endField) =>
@@ -473,6 +475,8 @@ case class Literal (value: Any, dataType: DataType) extends 
LeafExpression {
   s"DATE '$toString'"
 case (v: Long, TimestampType) =>
   s"TIMESTAMP '$toString'"
+case (v: Long, TimestampNTZType) =>
+  s"TIMESTAMP_NTZ '$toString'"
 case (i: CalendarInterval, CalendarIntervalType) =>
   s"INTERVAL '${i.toString}'"
 case (v: Array[Byte], BinaryType) => 
s"X'${DatatypeConverter.printHexBinary(v)}'"
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
index 50b7263..4081e13 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
@@ -362,6 +362,15 @@ class LiteralExpressionSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 }
   }
 
+  test("SPARK-36055: TimestampNTZ toString") {
+assert(Literal.default(TimestampNTZType).toString === "1970-01-01 
00:00:00")
+withTimeZones(sessionTimeZone = "GMT+01:00", systemTimeZone = "GMT-08:00") 
{
+  val timestamp = LocalDateTime.of(2021, 2, 3, 16, 50, 3, 45600)
+  val literalStr = Literal.create(timestamp).toString
+  assert(literalStr === "2021-02-03 16:50:03.456")
+}
+  }
+
   test("SPARK-35664: construct literals from java.time.LocalDateTime&

[spark] branch master updated (819c482 -> 382b66e)

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 819c482  [SPARK-35340][PYTHON] Standardize TypeError messages for 
unsupported basic operations
 add 382b66e  [SPARK-36054][SQL] Support group by TimestampNTZ type column

No new revisions were added by this update.

Summary of changes:
 .../org/apache/spark/sql/catalyst/expressions/hash.scala|  2 +-
 .../spark/sql/execution/vectorized/OffHeapColumnVector.java |  3 ++-
 .../spark/sql/execution/vectorized/OnHeapColumnVector.java  |  3 ++-
 .../spark/sql/execution/aggregate/HashMapGenerator.scala|  2 +-
 .../org/apache/spark/sql/DataFrameAggregateSuite.scala  | 13 -
 5 files changed, 18 insertions(+), 5 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36054][SQL] Support group by TimestampNTZ type column

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new ae62c9d  [SPARK-36054][SQL] Support group by TimestampNTZ type column
ae62c9d is described below

commit ae62c9d7726e2b05897f7e807bb9cdcc2748e3fa
Author: Gengliang Wang 
AuthorDate: Thu Jul 8 22:33:25 2021 +0300

[SPARK-36054][SQL] Support group by TimestampNTZ type column

### What changes were proposed in this pull request?

Support group by TimestampNTZ type column

### Why are the changes needed?

It's a basic SQL operation.

### Does this PR introduce _any_ user-facing change?

No, the new timestmap type is not released yet.

### How was this patch tested?

Unit test

Closes #33268 from gengliangwang/agg.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 382b66e26725e4667607303ebb9803b05e8076bc)
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/catalyst/expressions/hash.scala|  2 +-
 .../spark/sql/execution/vectorized/OffHeapColumnVector.java |  3 ++-
 .../spark/sql/execution/vectorized/OnHeapColumnVector.java  |  3 ++-
 .../spark/sql/execution/aggregate/HashMapGenerator.scala|  2 +-
 .../org/apache/spark/sql/DataFrameAggregateSuite.scala  | 13 -
 5 files changed, 18 insertions(+), 5 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
index d730586..3785262 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
@@ -490,7 +490,7 @@ abstract class HashExpression[E] extends Expression {
 case BooleanType => genHashBoolean(input, result)
 case ByteType | ShortType | IntegerType | DateType => genHashInt(input, 
result)
 case LongType => genHashLong(input, result)
-case TimestampType => genHashTimestamp(input, result)
+case TimestampType | TimestampNTZType => genHashTimestamp(input, result)
 case FloatType => genHashFloat(input, result)
 case DoubleType => genHashDouble(input, result)
 case d: DecimalType => genHashDecimal(ctx, d, input, result)
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
index 7da5a28..b4b6903 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java
@@ -553,7 +553,8 @@ public final class OffHeapColumnVector extends 
WritableColumnVector {
 type instanceof DateType || DecimalType.is32BitDecimalType(type)) {
   this.data = Platform.reallocateMemory(data, oldCapacity * 4L, 
newCapacity * 4L);
 } else if (type instanceof LongType || type instanceof DoubleType ||
-DecimalType.is64BitDecimalType(type) || type instanceof TimestampType) 
{
+DecimalType.is64BitDecimalType(type) || type instanceof TimestampType 
||
+type instanceof TimestampNTZType) {
   this.data = Platform.reallocateMemory(data, oldCapacity * 8L, 
newCapacity * 8L);
 } else if (childColumns != null) {
   // Nothing to store.
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java
index 5942c5f..3fb96d8 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java
@@ -547,7 +547,8 @@ public final class OnHeapColumnVector extends 
WritableColumnVector {
 if (intData != null) System.arraycopy(intData, 0, newData, 0, 
capacity);
 intData = newData;
   }
-} else if (type instanceof LongType || type instanceof TimestampType ||
+} else if (type instanceof LongType ||
+type instanceof TimestampType ||type instanceof TimestampNTZType ||
 DecimalType.is64BitDecimalType(type) || type instanceof 
DayTimeIntervalType) {
   if (longData == null || longData.length < newCapacity) {
 long[] newData = new long[newCapacity];
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index b3f5e34..713e7db 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
++

[spark] branch master updated (382b66e -> fef7e170)

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 382b66e  [SPARK-36054][SQL] Support group by TimestampNTZ type column
 add fef7e170 [SPARK-36049][SQL] Remove IntervalUnit

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/util/IntervalUtils.scala| 120 -
 .../catalyst/parser/ExpressionParserSuite.scala|  33 ++
 2 files changed, 58 insertions(+), 95 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36049][SQL] Remove IntervalUnit

2021-07-08 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 2f54d9e  [SPARK-36049][SQL] Remove IntervalUnit
2f54d9e is described below

commit 2f54d9eed6dce5e9fc853ef4dceb1abb2b338a34
Author: Angerszh 
AuthorDate: Thu Jul 8 23:02:21 2021 +0300

[SPARK-36049][SQL] Remove IntervalUnit

### What changes were proposed in this pull request?
Remove IntervalUnit

### Why are the changes needed?
Clean code

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Not need

Closes #33265 from AngersZh/SPARK-36049.

Lead-authored-by: Angerszh 
Co-authored-by: Maxim Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit fef7e1703c342165000f89b01112a8a28a936436)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala| 120 -
 .../catalyst/parser/ExpressionParserSuite.scala|  33 ++
 2 files changed, 58 insertions(+), 95 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 7579a28..e026266 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
@@ -41,22 +41,6 @@ object IntervalStringStyles extends Enumeration {
 
 object IntervalUtils {
 
-  object IntervalUnit extends Enumeration {
-type IntervalUnit = Value
-
-val NANOSECOND = Value(0, "nanosecond")
-val MICROSECOND = Value(1, "microsecond")
-val MILLISECOND = Value(2, "millisecond")
-val SECOND = Value(3, "second")
-val MINUTE = Value(4, "minute")
-val HOUR = Value(5, "hour")
-val DAY = Value(6, "day")
-val WEEK = Value(7, "week")
-val MONTH = Value(8, "month")
-val YEAR = Value(9, "year")
-  }
-  import IntervalUnit._
-
   private val MAX_DAY = Long.MaxValue / MICROS_PER_DAY
   private val MAX_HOUR = Long.MaxValue / MICROS_PER_HOUR
   private val MAX_MINUTE = Long.MaxValue / MICROS_PER_MINUTE
@@ -97,7 +81,7 @@ object IntervalUtils {
   def getSeconds(interval: CalendarInterval): Decimal = 
getSeconds(interval.microseconds)
 
   private def toLongWithRange(
-  fieldName: IntervalUnit,
+  fieldName: UTF8String,
   s: String,
   minValue: Long,
   maxValue: Long): Long = {
@@ -250,10 +234,11 @@ object IntervalUtils {
 }
   }
 
-  private def toYMInterval(yearStr: String, monthStr: String, sign: Int): Int 
= {
+  private def toYMInterval(year: String, month: String, sign: Int): Int = {
 safeToInterval("year-month") {
-  val years = toLongWithRange(YEAR, yearStr, 0, Integer.MAX_VALUE / 
MONTHS_PER_YEAR)
-  val totalMonths = sign * (years * MONTHS_PER_YEAR + 
toLongWithRange(MONTH, monthStr, 0, 11))
+  val years = toLongWithRange(yearStr, year, 0, Integer.MAX_VALUE / 
MONTHS_PER_YEAR)
+  val totalMonths =
+sign * (years * MONTHS_PER_YEAR + toLongWithRange(monthStr, month, 0, 
11))
   Math.toIntExact(totalMonths)
 }
   }
@@ -402,45 +387,33 @@ object IntervalUtils {
 }
   }
 
-  def toDTInterval(
-  dayStr: String,
-  hourStr: String,
-  minuteStr: String,
-  secondStr: String,
-  sign: Int): Long = {
+  def toDTInterval(day: String, hour: String, minute: String, second: String, 
sign: Int): Long = {
 var micros = 0L
-val days = toLongWithRange(DAY, dayStr, 0, MAX_DAY).toInt
+val days = toLongWithRange(dayStr, day, 0, MAX_DAY).toInt
 micros = Math.addExact(micros, sign * days * MICROS_PER_DAY)
-val hours = toLongWithRange(HOUR, hourStr, 0, 23)
+val hours = toLongWithRange(hourStr, hour, 0, 23)
 micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
-val minutes = toLongWithRange(MINUTE, minuteStr, 0, 59)
+val minutes = toLongWithRange(minuteStr, minute, 0, 59)
 micros = Math.addExact(micros, sign * minutes * MICROS_PER_MINUTE)
-micros = Math.addExact(micros, sign * parseSecondNano(secondStr))
+micros = Math.addExact(micros, sign * parseSecondNano(second))
 micros
   }
 
-  def toDTInterval(
-  hourStr: String,
-  minuteStr: String,
-  secondStr: String,
-  sign: Int): Long = {
+  def toDTInterval(hour: String, minute: String, second: String, sign: Int): 
Long = {
 var micros = 0L
-val hours = toLongWithRange(HOUR, hourStr, 0, MAX_HOUR)
+val hours = toLongWithRange(hourStr, hour, 0, MAX_HOUR)
 micros = Math.addExact(micros, sign * hours * MICROS_PER_HOUR)
-val minutes = toLongWithRange(MINU

[spark] branch master updated (cfcd094 -> 17ddcc9)

2021-07-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from cfcd094  [SPARK-36036][CORE] Fix cleanup of DownloadFile resources
 add 17ddcc9  [SPARK-36083][SQL] make_timestamp: return different result 
based on the default timestamp type

No new revisions were added by this update.

Summary of changes:
 .../catalyst/expressions/datetimeExpressions.scala |  30 --
 .../expressions/DateExpressionsSuite.scala | 120 +++--
 .../test/resources/sql-tests/inputs/datetime.sql   |   4 +
 .../sql-tests/results/ansi/datetime.sql.out|  19 +++-
 .../sql-tests/results/datetime-legacy.sql.out  |  18 +++-
 .../resources/sql-tests/results/datetime.sql.out   |  18 +++-
 .../results/timestampNTZ/datetime.sql.out  |  18 +++-
 7 files changed, 161 insertions(+), 66 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36083][SQL] make_timestamp: return different result based on the default timestamp type

2021-07-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 09e5bbd  [SPARK-36083][SQL] make_timestamp: return different result 
based on the default timestamp type
09e5bbd is described below

commit 09e5bbdfbe53ccb4efd85f5774b7bee9e731a14f
Author: Gengliang Wang 
AuthorDate: Sun Jul 11 20:47:49 2021 +0300

[SPARK-36083][SQL] make_timestamp: return different result based on the 
default timestamp type

### What changes were proposed in this pull request?

The SQL function MAKE_TIMESTAMP should return different results based on 
the default timestamp type:
* when "spark.sql.timestampType" is TIMESTAMP_NTZ, return TimestampNTZType 
literal
* when "spark.sql.timestampType" is TIMESTAMP_LTZ, return TimestampType 
literal

### Why are the changes needed?

As "spark.sql.timestampType" sets the default timestamp type, the 
make_timestamp function should behave consistently with it.

### Does this PR introduce _any_ user-facing change?

Yes, when the value of "spark.sql.timestampType" is TIMESTAMP_NTZ, the 
result type of `MAKE_TIMESTAMP` is of TIMESTAMP_NTZ type.

### How was this patch tested?

Unit test

Closes #33290 from gengliangwang/mkTS.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 17ddcc9e8273a098b63984b950bfa6cd36b58b99)
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/datetimeExpressions.scala |  30 --
 .../expressions/DateExpressionsSuite.scala | 120 +++--
 .../test/resources/sql-tests/inputs/datetime.sql   |   4 +
 .../sql-tests/results/ansi/datetime.sql.out|  19 +++-
 .../sql-tests/results/datetime-legacy.sql.out  |  18 +++-
 .../resources/sql-tests/results/datetime.sql.out   |  18 +++-
 .../results/timestampNTZ/datetime.sql.out  |  18 +++-
 7 files changed, 161 insertions(+), 66 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 be527ce..979eeba 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
@@ -2286,7 +2286,8 @@ case class MakeDate(
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
-  usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create 
timestamp from year, month, day, hour, min, sec and timezone fields.",
+  usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create 
timestamp from year, month, day, hour, min, sec and timezone fields. " +
+"The result data type is consistent with the value of configuration 
`spark.sql.timestampType`",
   arguments = """
 Arguments:
   * year - the year to represent, from 1 to 
@@ -2324,7 +2325,8 @@ case class MakeTimestamp(
 sec: Expression,
 timezone: Option[Expression] = None,
 timeZoneId: Option[String] = None,
-failOnError: Boolean = SQLConf.get.ansiEnabled)
+failOnError: Boolean = SQLConf.get.ansiEnabled,
+override val dataType: DataType = SQLConf.get.timestampType)
   extends SeptenaryExpression with TimeZoneAwareExpression with 
ImplicitCastInputTypes
 with NullIntolerant {
 
@@ -2335,7 +2337,8 @@ case class MakeTimestamp(
   hour: Expression,
   min: Expression,
   sec: Expression) = {
-this(year, month, day, hour, min, sec, None, None, SQLConf.get.ansiEnabled)
+this(year, month, day, hour, min, sec, None, None, SQLConf.get.ansiEnabled,
+  SQLConf.get.timestampType)
   }
 
   def this(
@@ -2346,7 +2349,8 @@ case class MakeTimestamp(
   min: Expression,
   sec: Expression,
   timezone: Expression) = {
-this(year, month, day, hour, min, sec, Some(timezone), None, 
SQLConf.get.ansiEnabled)
+this(year, month, day, hour, min, sec, Some(timezone), None, 
SQLConf.get.ansiEnabled,
+  SQLConf.get.timestampType)
   }
 
   override def children: Seq[Expression] = Seq(year, month, day, hour, min, 
sec) ++ timezone
@@ -2355,7 +2359,6 @@ case class MakeTimestamp(
   override def inputTypes: Seq[AbstractDataType] =
 Seq(IntegerType, IntegerType, IntegerType, IntegerType, IntegerType, 
DecimalType(8, 6)) ++
 timezone.map(_ => StringType)
-  override def dataType: DataType = TimestampType
   override def nullable: Boolean = if (failOnError) 
children.exists(_.nullable) else true
 
   override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
@@ -2388,7 +2391,11 @@ case class MakeTimestam

[spark] branch master updated (badb039 -> 8738682)

2021-07-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from badb039  [SPARK-36003][PYTHON] Implement unary operator `invert` of 
integral ps.Series/Index
 add 8738682  [SPARK-36044][SQL] Suport TimestampNTZ in functions 
unix_timestamp/to_unix_timestamp

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/expressions/datetimeExpressions.scala  |  8 +---
 .../spark/sql/catalyst/expressions/DateExpressionsSuite.scala |  9 +
 .../test/scala/org/apache/spark/sql/DateFunctionsSuite.scala  | 11 ++-
 3 files changed, 24 insertions(+), 4 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36044][SQL] Suport TimestampNTZ in functions unix_timestamp/to_unix_timestamp

2021-07-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 5816482  [SPARK-36044][SQL] Suport TimestampNTZ in functions 
unix_timestamp/to_unix_timestamp
5816482 is described below

commit 58164828683db50fb0e1698b679ffc0a773da847
Author: gengjiaan 
AuthorDate: Mon Jul 12 09:55:43 2021 +0300

[SPARK-36044][SQL] Suport TimestampNTZ in functions 
unix_timestamp/to_unix_timestamp

### What changes were proposed in this pull request?
The functions `unix_timestamp`/`to_unix_timestamp` should be able to accept 
input of `TimestampNTZType`.

### Why are the changes needed?
The functions `unix_timestamp`/`to_unix_timestamp` should be able to accept 
input of `TimestampNTZType`.

### Does this PR introduce _any_ user-facing change?
'Yes'.

### How was this patch tested?
New tests.

Closes #33278 from beliefer/SPARK-36044.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
(cherry picked from commit 8738682f6a36436da0e9fc332d58b2f41309e2c2)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/expressions/datetimeExpressions.scala  |  8 +---
 .../spark/sql/catalyst/expressions/DateExpressionsSuite.scala |  9 +
 .../test/scala/org/apache/spark/sql/DateFunctionsSuite.scala  | 11 ++-
 3 files changed, 24 insertions(+), 4 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 979eeba..f0ed89e 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
@@ -1091,8 +1091,10 @@ abstract class ToTimestamp
   override protected def formatString: Expression = right
   override protected def isParsing = true
 
+  override def forTimestampNTZ: Boolean = left.dataType == TimestampNTZType
+
   override def inputTypes: Seq[AbstractDataType] =
-Seq(TypeCollection(StringType, DateType, TimestampType), StringType)
+Seq(TypeCollection(StringType, DateType, TimestampType, TimestampNTZType), 
StringType)
 
   override def dataType: DataType = LongType
   override def nullable: Boolean = if (failOnError) 
children.exists(_.nullable) else true
@@ -1112,7 +1114,7 @@ abstract class ToTimestamp
   left.dataType match {
 case DateType =>
   daysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor
-case TimestampType =>
+case TimestampType | TimestampNTZType =>
   t.asInstanceOf[Long] / downScaleFactor
 case StringType =>
   val fmt = right.eval(input)
@@ -1192,7 +1194,7 @@ abstract class ToTimestamp
  |}
  |""".stripMargin)
   }
-  case TimestampType =>
+  case TimestampType | TimestampNTZType =>
 val eval1 = left.genCode(ctx)
 ev.copy(code = code"""
   ${eval1.code}
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 5f071c3..02d6d95 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
@@ -916,6 +916,11 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 Literal(new Timestamp(100)), Literal("-MM-dd 
HH:mm:ss"), timeZoneId),
   1000L)
 checkEvaluation(
+  UnixTimestamp(
+
Literal(DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(100))),
+Literal("-MM-dd HH:mm:ss"), timeZoneId),
+  1000L)
+checkEvaluation(
   UnixTimestamp(Literal(date1), Literal("-MM-dd HH:mm:ss"), 
timeZoneId),
   MICROSECONDS.toSeconds(
 DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), 
tz.toZoneId)))
@@ -981,6 +986,10 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
 checkEvaluation(ToUnixTimestamp(
   Literal(new Timestamp(100)), Literal(fmt1)),
   1000L)
+checkEvaluation(ToUnixTimestamp(
+  
Literal(DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(100))),
+  Literal(fmt1)),
+  1000L)
 checkEvaluation(
   ToUnixTimestamp(Literal(date1), L

[spark] branch master updated (8738682 -> 32720dd)

2021-07-12 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 8738682  [SPARK-36044][SQL] Suport TimestampNTZ in functions 
unix_timestamp/to_unix_timestamp
 add 32720dd  [SPARK-36072][SQL] TO_TIMESTAMP: return different results 
based on the default timestamp type

No new revisions were added by this update.

Summary of changes:
 .../catalyst/expressions/datetimeExpressions.scala | 54 +-
 .../expressions/DateExpressionsSuite.scala | 23 +++---
 .../results/timestampNTZ/datetime.sql.out  | 84 +++---
 3 files changed, 76 insertions(+), 85 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36072][SQL] TO_TIMESTAMP: return different results based on the default timestamp type

2021-07-12 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 4e9e2f3  [SPARK-36072][SQL] TO_TIMESTAMP: return different results 
based on the default timestamp type
4e9e2f3 is described below

commit 4e9e2f32e84cd1d317e25b85eb18f2c662bc641f
Author: Gengliang Wang 
AuthorDate: Mon Jul 12 10:12:30 2021 +0300

[SPARK-36072][SQL] TO_TIMESTAMP: return different results based on the 
default timestamp type

### What changes were proposed in this pull request?

The SQL function TO_TIMESTAMP should return different results based on the 
default timestamp type:
* when "spark.sql.timestampType" is TIMESTAMP_NTZ, return TimestampNTZType 
literal
* when "spark.sql.timestampType" is TIMESTAMP_LTZ, return TimestampType 
literal

This PR also refactor the class GetTimestamp and GetTimestampNTZ to reduce 
duplicated code.

### Why are the changes needed?

As "spark.sql.timestampType" sets the default timestamp type, the 
to_timestamp function should behave consistently with it.

### Does this PR introduce _any_ user-facing change?

Yes, when the value of "spark.sql.timestampType" is TIMESTAMP_NTZ, the 
result type of `TO_TIMESTAMP` is of TIMESTAMP_NTZ type.

### How was this patch tested?

Unit test

Closes #33280 from gengliangwang/to_timestamp.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 32720dd3e18ea43c6d88125a52356f40f808b300)
Signed-off-by: Max Gekk 
---
 .../catalyst/expressions/datetimeExpressions.scala | 54 +-
 .../expressions/DateExpressionsSuite.scala | 23 +++---
 .../results/timestampNTZ/datetime.sql.out  | 84 +++---
 3 files changed, 76 insertions(+), 85 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 f0ed89e..0ebeacb 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
@@ -1008,17 +1008,17 @@ case class UnixTimestamp(
 copy(timeExp = newLeft, format = newRight)
 }
 
-case class GetTimestampNTZ(
+/**
+ * Gets a timestamp from a string or a date.
+ */
+case class GetTimestamp(
 left: Expression,
 right: Expression,
+override val dataType: DataType,
 timeZoneId: Option[String] = None,
 failOnError: Boolean = SQLConf.get.ansiEnabled) extends ToTimestamp {
 
-  override val forTimestampNTZ: Boolean = true
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType)
-
-  override def dataType: DataType = TimestampNTZType
+  override val forTimestampNTZ: Boolean = dataType == TimestampNTZType
 
   override protected def downScaleFactor: Long = 1
 
@@ -1064,7 +1064,7 @@ case class ParseToTimestampNTZ(
 child: Expression) extends RuntimeReplaceable {
 
   def this(left: Expression, format: Expression) = {
-this(left, Option(format), GetTimestampNTZ(left, format))
+this(left, Option(format), GetTimestamp(left, format, TimestampNTZType))
   }
 
   def this(left: Expression) = this(left, None, Cast(left, TimestampNTZType))
@@ -1886,7 +1886,7 @@ case class ParseToDate(left: Expression, format: 
Option[Expression], child: Expr
   extends RuntimeReplaceable {
 
   def this(left: Expression, format: Expression) = {
-this(left, Option(format), Cast(GetTimestamp(left, format), DateType))
+this(left, Option(format), Cast(GetTimestamp(left, format, TimestampType), 
DateType))
   }
 
   def this(left: Expression) = {
@@ -1911,7 +1911,8 @@ case class ParseToDate(left: Expression, format: 
Option[Expression], child: Expr
   usage = """
 _FUNC_(timestamp_str[, fmt]) - Parses the `timestamp_str` 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.
+  a timestamp if the `fmt` is omitted. The result data type is consistent 
with the value of
+  configuration `spark.sql.timestampType`.
   """,
   arguments = """
 Arguments:
@@ -1929,20 +1930,24 @@ case class ParseToDate(left: Expression, format: 
Option[Expression], child: Expr
   group = "datetime_funcs",
   since = "2.2.0")
 // scalastyle:on line.size.limit
-case class ParseToTimestamp(left: Expression, format: Option[Expression], 
child: Expression)
-  extends RuntimeReplaceable {
+case class ParseToTimestamp(
+left: Expression,
+format: Opt

[spark] branch master updated: [SPARK-36046][SQL] Support new functions make_timestamp_ntz and make_timestamp_ltz

2021-07-12 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 92bf83e  [SPARK-36046][SQL] Support new functions make_timestamp_ntz 
and make_timestamp_ltz
92bf83e is described below

commit 92bf83ed0aba2c399debb1db5fb88bad3961ab06
Author: Gengliang Wang 
AuthorDate: Mon Jul 12 22:44:26 2021 +0300

[SPARK-36046][SQL] Support new functions make_timestamp_ntz and 
make_timestamp_ltz

### What changes were proposed in this pull request?

Support new functions make_timestamp_ntz and make_timestamp_ltz
Syntax:
* `make_timestamp_ntz(year, month, day, hour, min, sec)`: Create local 
date-time from year, month, day, hour, min, sec fields
* `make_timestamp_ltz(year, month, day, hour, min, sec[, timezone])`: 
Create current timestamp with local time zone from year, month, day, hour, min, 
sec and timezone fields

### Why are the changes needed?

As the result of `make_timestamp` become consistent with the SQL 
configuration `spark.sql.timestmapType`, we need these two new functions to 
construct timestamp literals. They align to the functions [`make_timestamp` and 
`make_timestamptz`](https://www.postgresql.org/docs/9.4/functions-datetime.html)
 in PostgreSQL

### Does this PR introduce _any_ user-facing change?

Yes, two new datetime functions: make_timestamp_ntz and make_timestamp_ltz.

### How was this patch tested?

End-to-end tests.

Closes #33299 from gengliangwang/make_timestamp_ntz_ltz.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/analysis/FunctionRegistry.scala   |   2 +
 .../catalyst/expressions/datetimeExpressions.scala | 122 +
 .../sql-functions/sql-expression-schema.md |   6 +-
 .../test/resources/sql-tests/inputs/datetime.sql   |  12 ++
 .../sql-tests/results/ansi/datetime.sql.out|  61 ++-
 .../sql-tests/results/datetime-legacy.sql.out  |  59 +-
 .../resources/sql-tests/results/datetime.sql.out   |  59 +-
 .../results/timestampNTZ/datetime.sql.out  |  59 +-
 8 files changed, 374 insertions(+), 6 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index fcc0220..4fd871d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -552,6 +552,8 @@ object FunctionRegistry {
 expression[TimeWindow]("window"),
 expression[MakeDate]("make_date"),
 expression[MakeTimestamp]("make_timestamp"),
+expression[MakeTimestampNTZ]("make_timestamp_ntz", true),
+expression[MakeTimestampLTZ]("make_timestamp_ltz", true),
 expression[MakeInterval]("make_interval"),
 expression[MakeDTInterval]("make_dt_interval"),
 expression[MakeYMInterval]("make_ym_interval"),
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 0ebeacb..2840b18 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
@@ -2272,6 +2272,128 @@ case class MakeDate(
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
+  usage = "_FUNC_(year, month, day, hour, min, sec) - Create local date-time 
from year, month, day, hour, min, sec fields. ",
+  arguments = """
+Arguments:
+  * year - the year to represent, from 1 to 
+  * month - the month-of-year to represent, from 1 (January) to 12 
(December)
+  * day - the day-of-month to represent, from 1 to 31
+  * hour - the hour-of-day to represent, from 0 to 23
+  * min - the minute-of-hour to represent, from 0 to 59
+  * sec - the second-of-minute and its micro-fraction to represent, from
+  0 to 60. If the sec argument equals to 60, the seconds field is 
set
+  to 0 and 1 minute is added to the final timestamp.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887);
+   2014-12-28 06:30:45.887
+  > SELECT _FUNC_(2019, 6, 30, 23, 59, 60);
+   2019-07-01 00:00:00
+  > SELECT _FUNC_(null, 7, 22, 15, 30, 0);
+   NULL
+  """,
+  group = "datetime_funcs",
+  since = "3.2.0")
+// sca

[spark] branch branch-3.2 updated: [SPARK-36046][SQL] Support new functions make_timestamp_ntz and make_timestamp_ltz

2021-07-12 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new fba3e90  [SPARK-36046][SQL] Support new functions make_timestamp_ntz 
and make_timestamp_ltz
fba3e90 is described below

commit fba3e90863e584361b2f61828a500c96d89c35de
Author: Gengliang Wang 
AuthorDate: Mon Jul 12 22:44:26 2021 +0300

[SPARK-36046][SQL] Support new functions make_timestamp_ntz and 
make_timestamp_ltz

### What changes were proposed in this pull request?

Support new functions make_timestamp_ntz and make_timestamp_ltz
Syntax:
* `make_timestamp_ntz(year, month, day, hour, min, sec)`: Create local 
date-time from year, month, day, hour, min, sec fields
* `make_timestamp_ltz(year, month, day, hour, min, sec[, timezone])`: 
Create current timestamp with local time zone from year, month, day, hour, min, 
sec and timezone fields

### Why are the changes needed?

As the result of `make_timestamp` become consistent with the SQL 
configuration `spark.sql.timestmapType`, we need these two new functions to 
construct timestamp literals. They align to the functions [`make_timestamp` and 
`make_timestamptz`](https://www.postgresql.org/docs/9.4/functions-datetime.html)
 in PostgreSQL

### Does this PR introduce _any_ user-facing change?

Yes, two new datetime functions: make_timestamp_ntz and make_timestamp_ltz.

### How was this patch tested?

End-to-end tests.

Closes #33299 from gengliangwang/make_timestamp_ntz_ltz.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 92bf83ed0aba2c399debb1db5fb88bad3961ab06)
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/analysis/FunctionRegistry.scala   |   2 +
 .../catalyst/expressions/datetimeExpressions.scala | 122 +
 .../sql-functions/sql-expression-schema.md |   6 +-
 .../test/resources/sql-tests/inputs/datetime.sql   |  12 ++
 .../sql-tests/results/ansi/datetime.sql.out|  61 ++-
 .../sql-tests/results/datetime-legacy.sql.out  |  59 +-
 .../resources/sql-tests/results/datetime.sql.out   |  59 +-
 .../results/timestampNTZ/datetime.sql.out  |  59 +-
 8 files changed, 374 insertions(+), 6 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index fcc0220..4fd871d 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -552,6 +552,8 @@ object FunctionRegistry {
 expression[TimeWindow]("window"),
 expression[MakeDate]("make_date"),
 expression[MakeTimestamp]("make_timestamp"),
+expression[MakeTimestampNTZ]("make_timestamp_ntz", true),
+expression[MakeTimestampLTZ]("make_timestamp_ltz", true),
 expression[MakeInterval]("make_interval"),
 expression[MakeDTInterval]("make_dt_interval"),
 expression[MakeYMInterval]("make_ym_interval"),
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 0ebeacb..2840b18 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
@@ -2272,6 +2272,128 @@ case class MakeDate(
 
 // scalastyle:off line.size.limit
 @ExpressionDescription(
+  usage = "_FUNC_(year, month, day, hour, min, sec) - Create local date-time 
from year, month, day, hour, min, sec fields. ",
+  arguments = """
+Arguments:
+  * year - the year to represent, from 1 to 
+  * month - the month-of-year to represent, from 1 (January) to 12 
(December)
+  * day - the day-of-month to represent, from 1 to 31
+  * hour - the hour-of-day to represent, from 0 to 23
+  * min - the minute-of-hour to represent, from 0 to 59
+  * sec - the second-of-minute and its micro-fraction to represent, from
+  0 to 60. If the sec argument equals to 60, the seconds field is 
set
+  to 0 and 1 minute is added to the final timestamp.
+  """,
+  examples = """
+Examples:
+  > SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887);
+   2014-12-28 06:30:45.887
+  > SELECT _FUNC_(2019, 6, 30, 23, 59, 60);
+   2019-07-01 00:00:00
+  > SELECT _FUNC_(null, 7, 22, 15, 30, 0);

[spark] branch master updated (03e48c8 -> 1ba3982)

2021-07-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 03e48c8  [SPARK-35334][K8S] Make Spark more resilient to intermittent 
K8s flakiness
 add 1ba3982  [SPARK-35735][SQL][FOLLOWUP] Remove unused method 
`IntervalUtils.checkIntervalStringDataType()`

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/util/IntervalUtils.scala | 21 +
 1 file changed, 1 insertion(+), 20 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-35735][SQL][FOLLOWUP] Remove unused method `IntervalUtils.checkIntervalStringDataType()`

2021-07-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new b469294  [SPARK-35735][SQL][FOLLOWUP] Remove unused method 
`IntervalUtils.checkIntervalStringDataType()`
b469294 is described below

commit b4692949f802a01bcdd7364235ce9e7ed31ff14d
Author: Max Gekk 
AuthorDate: Tue Jul 13 15:11:21 2021 +0300

[SPARK-35735][SQL][FOLLOWUP] Remove unused method 
`IntervalUtils.checkIntervalStringDataType()`

### What changes were proposed in this pull request?
Remove the private method `checkIntervalStringDataType()` from 
`IntervalUtils` since it hasn't been used anymore after 
https://github.com/apache/spark/pull/33242.

### Why are the changes needed?
To improve code maintenance.

### Does this PR introduce _any_ user-facing change?
No. The method is private, and it existing in code base for short time.

### How was this patch tested?
By existing GAs/tests.

Closes #33321 from MaxGekk/SPARK-35735-remove-unused-method.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit 1ba3982d16f98601583520794b30fa6ad6d85cf0)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/util/IntervalUtils.scala | 21 +
 1 file changed, 1 insertion(+), 20 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 e026266..dc6c02e 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,7 +30,7 @@ import 
org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DataType, DayTimeIntervalType => DT, 
Decimal, YearMonthIntervalType => YM}
+import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, 
YearMonthIntervalType => YM}
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 // The style of textual representation of intervals
@@ -124,25 +124,6 @@ object IntervalUtils {
 s"${fallBackNotice.map(s => s", $s").getOrElse("")}")
   }
 
-  private def checkIntervalStringDataType(
-  input: UTF8String,
-  targetStartField: Byte,
-  targetEndField: Byte,
-  inputIntervalType: DataType,
-  fallBackNotice: Option[String] = None): Unit = {
-val (intervalStr, typeName, inputStartField, inputEndField) = 
inputIntervalType match {
-  case DT(startField, endField) =>
-("day-time", DT(targetStartField, targetEndField).typeName, 
startField, endField)
-  case YM(startField, endField) =>
-("year-month", YM(targetStartField, targetEndField).typeName, 
startField, endField)
-}
-if (targetStartField != inputStartField || targetEndField != 
inputEndField) {
-  throwIllegalIntervalFormatException(
-input, targetStartField, targetEndField, intervalStr, typeName, 
fallBackNotice)
-}
-  }
-
-
   val supportedFormat = Map(
 (YM.YEAR, YM.MONTH) -> Seq("[+|-]y-m", "INTERVAL [+|-]'[+|-]y-m' YEAR TO 
MONTH"),
 (YM.YEAR, YM.YEAR) -> Seq("[+|-]y", "INTERVAL [+|-]'[+|-]y' YEAR"),

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



[spark] branch master updated (583173b -> 0674327)

2021-07-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 583173b  [SPARK-36033][SQL][TEST] Validate partitioning requirements 
in TPCDS tests
 add 0674327  [SPARK-36120][SQL] Support TimestampNTZ type in cache table

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/execution/columnar/ColumnBuilder.scala  |  3 ++-
 .../apache/spark/sql/execution/columnar/ColumnType.scala  |  2 +-
 .../sql/execution/columnar/GenerateColumnAccessor.scala   |  2 +-
 .../scala/org/apache/spark/sql/CachedTableSuite.scala | 15 ++-
 4 files changed, 18 insertions(+), 4 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36120][SQL] Support TimestampNTZ type in cache table

2021-07-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 3ace01b  [SPARK-36120][SQL] Support TimestampNTZ type in cache table
3ace01b is described below

commit 3ace01b25bf1c69901a07c11acbae6fa996fc06c
Author: Gengliang Wang 
AuthorDate: Tue Jul 13 17:23:48 2021 +0300

[SPARK-36120][SQL] Support TimestampNTZ type in cache table

### What changes were proposed in this pull request?

Support TimestampNTZ type column in SQL command Cache table

### Why are the changes needed?

Cache table should support the new timestamp type.

### Does this PR introduce _any_ user-facing change?

Yes, the TimemstampNTZ type column can used in `CACHE TABLE`

### How was this patch tested?

Unit test

Closes #33322 from gengliangwang/cacheTable.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 067432705fbec970bd713adf37d2aa17c6bcf5a0)
Signed-off-by: Max Gekk 
---
 .../spark/sql/execution/columnar/ColumnBuilder.scala  |  3 ++-
 .../apache/spark/sql/execution/columnar/ColumnType.scala  |  2 +-
 .../sql/execution/columnar/GenerateColumnAccessor.scala   |  2 +-
 .../scala/org/apache/spark/sql/CachedTableSuite.scala | 15 ++-
 4 files changed, 18 insertions(+), 4 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
index e9251e8..9ddc665 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBuilder.scala
@@ -175,7 +175,8 @@ private[columnar] object ColumnBuilder {
   case ByteType => new ByteColumnBuilder
   case ShortType => new ShortColumnBuilder
   case IntegerType | DateType | _: YearMonthIntervalType => new 
IntColumnBuilder
-  case LongType | TimestampType | _: DayTimeIntervalType => new 
LongColumnBuilder
+  case LongType | TimestampType | TimestampNTZType | _: 
DayTimeIntervalType =>
+new LongColumnBuilder
   case FloatType => new FloatColumnBuilder
   case DoubleType => new DoubleColumnBuilder
   case StringType => new StringColumnBuilder
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
index 8e99368..419dcc6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
@@ -818,7 +818,7 @@ private[columnar] object ColumnType {
   case ByteType => BYTE
   case ShortType => SHORT
   case IntegerType | DateType | _: YearMonthIntervalType => INT
-  case LongType | TimestampType | _: DayTimeIntervalType => LONG
+  case LongType | TimestampType | TimestampNTZType | _: 
DayTimeIntervalType => LONG
   case FloatType => FLOAT
   case DoubleType => DOUBLE
   case StringType => STRING
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
index 190c2c3..6e666d4 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/GenerateColumnAccessor.scala
@@ -81,7 +81,7 @@ object GenerateColumnAccessor extends 
CodeGenerator[Seq[DataType], ColumnarItera
 case ByteType => classOf[ByteColumnAccessor].getName
 case ShortType => classOf[ShortColumnAccessor].getName
 case IntegerType | DateType | _: YearMonthIntervalType => 
classOf[IntColumnAccessor].getName
-case LongType | TimestampType | _: DayTimeIntervalType =>
+case LongType | TimestampType | TimestampNTZType | _: 
DayTimeIntervalType =>
   classOf[LongColumnAccessor].getName
 case FloatType => classOf[FloatColumnAccessor].getName
 case DoubleType => classOf[DoubleColumnAccessor].getName
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
index 1915044..db717d2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql
 
 import java.io.{File, FilenameFilter}
 import java.nio.file.{Files, Paths}
-import java.time.{Duration, 

[spark] branch master updated (38196412 -> 1e86345)

2021-07-14 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 38196412 [SPARK-35639][SQL][FOLLOWUP] Make hasCoalescedPartition 
return true if something was actually coalesced
 add 1e86345  [SPARK-36069][SQL] Add field info to `from_json`'s exception 
in the FAILFAST mode

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/json/JacksonParser.scala| 10 -
 .../spark/sql/errors/QueryExecutionErrors.scala| 17 ++-
 .../org/apache/spark/sql/JsonFunctionsSuite.scala  | 25 ++
 3 files changed, 42 insertions(+), 10 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on daylight saving time

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 75ff69a  [SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on 
daylight saving time
75ff69a is described below

commit 75ff69a9940776f0337a080b721d45d9f571c386
Author: Gengliang Wang 
AuthorDate: Thu Jul 15 11:40:51 2021 +0300

[SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on daylight saving 
time

### What changes were proposed in this pull request?

Only use the zone ids that has no daylight saving for testing 
`localtimestamp`

### Why are the changes needed?

https://github.com/apache/spark/pull/33346#discussion_r670135296 MaxGekk 
suggests that we should avoid wrong results if possible.

### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

Unit test

Closes #33354 from gengliangwang/FIxDST.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 564d3de7c62fa89c6db1e08e809400b339a704cd)
Signed-off-by: Max Gekk 
---
 .../apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

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 dbfe0af..eecaa93 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
@@ -97,7 +97,8 @@ class DateExpressionsSuite extends SparkFunSuite with 
ExpressionEvalHelper {
   }
 
   test("datetime function localtimestamp") {
-outstandingTimezonesIds.foreach { zid =>
+// Verify with multiple outstanding time zones which has no daylight 
saving time.
+Seq("UTC", "Africa/Dakar", "Asia/Hong_Kong").foreach { zid =>
   val zoneId = DateTimeUtils.getZoneId(zid)
   val ct = LocalTimestamp(Some(zid)).eval(EmptyRow).asInstanceOf[Long]
   val t1 = DateTimeUtils.localDateTimeToMicros(LocalDateTime.now(zoneId))

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



[spark] branch master updated (5acfecb -> 564d3de)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 5acfecb  [SPARK-36150][INFRA][TESTS] Disable MiMa for Scala 2.13 
artifacts
 add 564d3de  [SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on 
daylight saving time

No new revisions were added by this update.

Summary of changes:
 .../apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

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



[spark] branch master updated (564d3de -> b5ee6d0)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 564d3de  [SPARK-36037][TESTS][FOLLOWUP] Avoid wrong test results on 
daylight saving time
 add b5ee6d0  [SPARK-36149][PYTHON] Clarify documentation for dayofweek and 
weekofyear

No new revisions were added by this update.

Summary of changes:
 R/pkg/R/functions.R | 3 +++
 python/pyspark/sql/functions.py | 3 +++
 2 files changed, 6 insertions(+)

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



[spark] branch master updated (b5ee6d0 -> 4dfd266)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from b5ee6d0  [SPARK-36149][PYTHON] Clarify documentation for dayofweek and 
weekofyear
 add 4dfd266  [SPARK-36148][SQL] Fix input data types check for 
regexp_replace

No new revisions were added by this update.

Summary of changes:
 .../apache/spark/sql/catalyst/expressions/regexpExpressions.scala  | 4 
 .../src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala | 7 +++
 2 files changed, 11 insertions(+)

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



[spark] branch master updated: [SPARK-36154][DOCS] Documenting week and quarter as valid formats in pyspark sql/functions trunc

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 802f632  [SPARK-36154][DOCS] Documenting week and quarter as valid 
formats in pyspark sql/functions trunc
802f632 is described below

commit 802f632a28e46538053c056d1ce43374f80454ae
Author: Dominik Gehl 
AuthorDate: Thu Jul 15 16:51:11 2021 +0300

[SPARK-36154][DOCS] Documenting week and quarter as valid formats in 
pyspark sql/functions trunc

### What changes were proposed in this pull request?
Added missing documentation of week and quarter as valid formats to pyspark 
sql/functions trunc

### Why are the changes needed?
Pyspark documentation and scala documentation didn't mentioned the same 
supported formats

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Only documentation change

Closes #33359 from dominikgehl/feature/SPARK-36154.

Authored-by: Dominik Gehl 
Signed-off-by: Max Gekk 
---
 R/pkg/R/functions.R |  9 ++---
 python/pyspark/sql/functions.py | 11 ---
 2 files changed, 14 insertions(+), 6 deletions(-)

diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 892a8dd..40642f1 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -47,10 +47,13 @@ NULL
 #'   \item \code{to_date} and \code{to_timestamp}: it is the 
string to use to parse
 #'Column \code{x} to DateType or TimestampType.
 #'   \item \code{trunc}: it is the string to use to specify the 
truncation method.
-#'For example, "year", "", "yy" for truncate by year, 
or "month", "mon",
-#'"mm" for truncate by month.
+#''year', '', 'yy' to truncate by year,
+#'or 'month', 'mon', 'mm' to truncate by month
+#'Other options are: 'week', 'quarter'
 #'   \item \code{date_trunc}: it is similar with \code{trunc}'s 
but additionally
-#'supports "day", "dd", "second", "minute", "hour", "week" 
and "quarter".
+#'supports
+#''day', 'dd' to truncate by day,
+#''microsecond', 'millisecond', 'second', 'minute' and 
'hour'
 #'   }
 #' @param ... additional argument(s).
 #' @name column_datetime_functions
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index bd32217..e5b2a00 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2045,7 +2045,9 @@ def trunc(date, format):
 --
 date : :class:`~pyspark.sql.Column` or str
 format : str
-'year', '', 'yy' or 'month', 'mon', 'mm'
+'year', '', 'yy' to truncate by year,
+or 'month', 'mon', 'mm' to truncate by month
+Other options are: 'week', 'quarter'
 
 Examples
 
@@ -2068,8 +2070,11 @@ def date_trunc(format, timestamp):
 Parameters
 --
 format : str
-'year', '', 'yy', 'month', 'mon', 'mm',
-'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
+'year', '', 'yy' to truncate by year,
+'month', 'mon', 'mm' to truncate by month,
+'day', 'dd' to truncate by day,
+Other options are:
+'microsecond', 'millisecond', 'second', 'minute', 'hour', 'week', 
'quarter'
 timestamp : :class:`~pyspark.sql.Column` or str
 
 Examples

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



[spark] branch branch-3.2 updated: [SPARK-36154][DOCS] Documenting week and quarter as valid formats in pyspark sql/functions trunc

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 3a09024  [SPARK-36154][DOCS] Documenting week and quarter as valid 
formats in pyspark sql/functions trunc
3a09024 is described below

commit 3a0902463680abeadb46d5bd6b99cfc72e836c8d
Author: Dominik Gehl 
AuthorDate: Thu Jul 15 16:51:11 2021 +0300

[SPARK-36154][DOCS] Documenting week and quarter as valid formats in 
pyspark sql/functions trunc

### What changes were proposed in this pull request?
Added missing documentation of week and quarter as valid formats to pyspark 
sql/functions trunc

### Why are the changes needed?
Pyspark documentation and scala documentation didn't mentioned the same 
supported formats

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Only documentation change

Closes #33359 from dominikgehl/feature/SPARK-36154.

Authored-by: Dominik Gehl 
Signed-off-by: Max Gekk 
(cherry picked from commit 802f632a28e46538053c056d1ce43374f80454ae)
Signed-off-by: Max Gekk 
---
 R/pkg/R/functions.R |  9 ++---
 python/pyspark/sql/functions.py | 11 ---
 2 files changed, 14 insertions(+), 6 deletions(-)

diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 656206f..2adff08 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -47,10 +47,13 @@ NULL
 #'   \item \code{to_date} and \code{to_timestamp}: it is the 
string to use to parse
 #'Column \code{x} to DateType or TimestampType.
 #'   \item \code{trunc}: it is the string to use to specify the 
truncation method.
-#'For example, "year", "", "yy" for truncate by year, 
or "month", "mon",
-#'"mm" for truncate by month.
+#''year', '', 'yy' to truncate by year,
+#'or 'month', 'mon', 'mm' to truncate by month
+#'Other options are: 'week', 'quarter'
 #'   \item \code{date_trunc}: it is similar with \code{trunc}'s 
but additionally
-#'supports "day", "dd", "second", "minute", "hour", "week" 
and "quarter".
+#'supports
+#''day', 'dd' to truncate by day,
+#''microsecond', 'millisecond', 'second', 'minute' and 
'hour'
 #'   }
 #' @param ... additional argument(s).
 #' @name column_datetime_functions
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 6079949..d4f527d 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -2042,7 +2042,9 @@ def trunc(date, format):
 --
 date : :class:`~pyspark.sql.Column` or str
 format : str
-'year', '', 'yy' or 'month', 'mon', 'mm'
+'year', '', 'yy' to truncate by year,
+or 'month', 'mon', 'mm' to truncate by month
+Other options are: 'week', 'quarter'
 
 Examples
 
@@ -2065,8 +2067,11 @@ def date_trunc(format, timestamp):
 Parameters
 --
 format : str
-'year', '', 'yy', 'month', 'mon', 'mm',
-'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
+'year', '', 'yy' to truncate by year,
+'month', 'mon', 'mm' to truncate by month,
+'day', 'dd' to truncate by day,
+Other options are:
+'microsecond', 'millisecond', 'second', 'minute', 'hour', 'week', 
'quarter'
 timestamp : :class:`~pyspark.sql.Column` or str
 
 Examples

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



[spark] branch branch-3.1 updated: [SPARK-36154][DOCS] Documenting week and quarter as valid formats in pyspark sql/functions trunc

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 249b567  [SPARK-36154][DOCS] Documenting week and quarter as valid 
formats in pyspark sql/functions trunc
249b567 is described below

commit 249b567684f977a1af676c4cec14641ae6c75269
Author: Dominik Gehl 
AuthorDate: Thu Jul 15 16:51:11 2021 +0300

[SPARK-36154][DOCS] Documenting week and quarter as valid formats in 
pyspark sql/functions trunc

### What changes were proposed in this pull request?
Added missing documentation of week and quarter as valid formats to pyspark 
sql/functions trunc

### Why are the changes needed?
Pyspark documentation and scala documentation didn't mentioned the same 
supported formats

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Only documentation change

Closes #33359 from dominikgehl/feature/SPARK-36154.

Authored-by: Dominik Gehl 
Signed-off-by: Max Gekk 
(cherry picked from commit 802f632a28e46538053c056d1ce43374f80454ae)
Signed-off-by: Max Gekk 
---
 R/pkg/R/functions.R |  9 ++---
 python/pyspark/sql/functions.py | 11 ---
 2 files changed, 14 insertions(+), 6 deletions(-)

diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R
index 28e4ef8..87a75c3 100644
--- a/R/pkg/R/functions.R
+++ b/R/pkg/R/functions.R
@@ -47,10 +47,13 @@ NULL
 #'   \item \code{to_date} and \code{to_timestamp}: it is the 
string to use to parse
 #'Column \code{x} to DateType or TimestampType.
 #'   \item \code{trunc}: it is the string to use to specify the 
truncation method.
-#'For example, "year", "", "yy" for truncate by year, 
or "month", "mon",
-#'"mm" for truncate by month.
+#''year', '', 'yy' to truncate by year,
+#'or 'month', 'mon', 'mm' to truncate by month
+#'Other options are: 'week', 'quarter'
 #'   \item \code{date_trunc}: it is similar with \code{trunc}'s 
but additionally
-#'supports "day", "dd", "second", "minute", "hour", "week" 
and "quarter".
+#'supports
+#''day', 'dd' to truncate by day,
+#''microsecond', 'millisecond', 'second', 'minute' and 
'hour'
 #'   }
 #' @param ... additional argument(s).
 #' @name column_datetime_functions
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 75a31b7..88b7c4d 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -1941,7 +1941,9 @@ def trunc(date, format):
 --
 date : :class:`~pyspark.sql.Column` or str
 format : str
-'year', '', 'yy' or 'month', 'mon', 'mm'
+'year', '', 'yy' to truncate by year,
+or 'month', 'mon', 'mm' to truncate by month
+Other options are: 'week', 'quarter'
 
 Examples
 
@@ -1964,8 +1966,11 @@ def date_trunc(format, timestamp):
 Parameters
 --
 format : str
-'year', '', 'yy', 'month', 'mon', 'mm',
-'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
+'year', '', 'yy' to truncate by year,
+'month', 'mon', 'mm' to truncate by month,
+'day', 'dd' to truncate by day,
+Other options are:
+'microsecond', 'millisecond', 'second', 'minute', 'hour', 'week', 
'quarter'
 timestamp : :class:`~pyspark.sql.Column` or str
 
 Examples

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



[spark] branch master updated (a71dd6a -> 0062c03)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from a71dd6a  [SPARK-36146][PYTHON][INFRA][TESTS] Upgrade Python version 
from 3.6 to 3.9 in GitHub Actions' linter/docs
 add 0062c03  [SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT 
IN predicate

No new revisions were added by this update.

Summary of changes:
 .../sql/execution/datasources/parquet/ParquetFilters.scala |  4 +++-
 .../execution/datasources/parquet/ParquetFilterSuite.scala | 14 ++
 2 files changed, 17 insertions(+), 1 deletion(-)

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



[spark] branch branch-3.2 updated: [SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT IN predicate

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 1ed72e2  [SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT 
IN predicate
1ed72e2 is described below

commit 1ed72e2e8e25991aee4cd8c50d627c5477a3
Author: Yuming Wang 
AuthorDate: Thu Jul 15 18:51:53 2021 +0300

[SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT IN predicate

### What changes were proposed in this pull request?

This pr fix Parquet filter pushdown `NOT` `IN` predicate if its values 
exceeds `spark.sql.parquet.pushdown.inFilterThreshold`. For example: `Not(In(a, 
Array(2, 3, 7))`. We can not push down `not(and(gteq(a, 2), lteq(a, 7)))`.

### Why are the changes needed?

Fix bug.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit test.

Closes #33365 from wangyum/SPARK-32792-3.

Authored-by: Yuming Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 0062c03c1584b0ab1443fa24f288e611bfbb5115)
Signed-off-by: Max Gekk 
---
 .../sql/execution/datasources/parquet/ParquetFilters.scala |  4 +++-
 .../execution/datasources/parquet/ParquetFilterSuite.scala | 14 ++
 2 files changed, 17 insertions(+), 1 deletion(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
index 71205f9..df7 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
@@ -710,7 +710,7 @@ class ParquetFilters(
   values.distinct.flatMap { v =>
 makeEq.lift(fieldType).map(_(fieldNames, v))
   }.reduceLeftOption(FilterApi.or)
-} else {
+} else if (canPartialPushDownConjuncts) {
   val primitiveType = 
schema.getColumnDescription(fieldNames).getPrimitiveType
   val statistics: ParquetStatistics[_] = 
ParquetStatistics.createStats(primitiveType)
   if (values.contains(null)) {
@@ -721,6 +721,8 @@ class ParquetFilters(
   } else {
 makeInPredicate.lift(fieldType).map(_(fieldNames, values, 
statistics))
   }
+} else {
+  None
 }
 
   case sources.StringStartsWith(name, prefix)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index bdcdbf3..230d547 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -1563,6 +1563,20 @@ abstract class ParquetFilterSuite extends QueryTest with 
ParquetTest with Shared
   ) {
 parquetFilters.createFilter(sources.In("a", Array(2, 3, 7, null, 6)))
   }
+
+  assertResult(
+Some(FilterApi.not(or(
+  FilterApi.eq(intColumn("a"), 2: Integer),
+  FilterApi.eq(intColumn("a"), 3: Integer
+  ) {
+parquetFilters.createFilter(sources.Not(sources.In("a", Array(2, 3
+  }
+
+  assertResult(
+None
+  ) {
+parquetFilters.createFilter(sources.Not(sources.In("a", Array(2, 3, 
7
+  }
 }
   }
 

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



[spark] branch master updated (0062c03 -> 2db7ed7)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 0062c03  [SPARK-32792][SQL][FOLLOWUP] Fix Parquet filter pushdown NOT 
IN predicate
 add 2db7ed7  [SPARK-36158][PYTHON][DOCS] Improving pyspark sql/functions 
months_between documentation

No new revisions were added by this update.

Summary of changes:
 python/pyspark/sql/functions.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

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



[spark] branch master updated (c8a3c22 -> b09b7f7)

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from c8a3c22  [SPARK-36164][INFRA] run-test.py should not fail when 
APACHE_SPARK_REF is not defined
 add b09b7f7  [SPARK-36034][SQL] Rebase datetime in pushed down filters to 
parquet

No new revisions were added by this update.

Summary of changes:
 .../datasources/parquet/ParquetFileFormat.scala|  17 ++-
 .../datasources/parquet/ParquetFilters.scala   |  29 +++-
 .../v2/parquet/ParquetPartitionReaderFactory.scala |  17 ++-
 .../v2/parquet/ParquetScanBuilder.scala|  14 +-
 .../datasources/parquet/ParquetFilterSuite.scala   | 164 +++--
 5 files changed, 145 insertions(+), 96 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36034][SQL] Rebase datetime in pushed down filters to parquet

2021-07-15 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 57849a5  [SPARK-36034][SQL] Rebase datetime in pushed down filters to 
parquet
57849a5 is described below

commit 57849a54da39c337fc7209404815e8980e83e03b
Author: Max Gekk 
AuthorDate: Thu Jul 15 22:21:57 2021 +0300

[SPARK-36034][SQL] Rebase datetime in pushed down filters to parquet

### What changes were proposed in this pull request?
In the PR, I propose to propagate either the SQL config 
`spark.sql.parquet.datetimeRebaseModeInRead` or/and Parquet option 
`datetimeRebaseMode` to `ParquetFilters`. The `ParquetFilters` class uses the 
settings in conversions of dates/timestamps instances from datasource filters 
to values pushed via `FilterApi` to the `parquet-column` lib.

Before the changes, date/timestamp values expressed as 
days/microseconds/milliseconds are interpreted as offsets in Proleptic 
Gregorian calendar, and pushed to the parquet library as is. That works fine if 
timestamp/dates values in parquet files were saved in the `CORRECTED` mode but 
in the `LEGACY` mode, filter's values could not match to actual values.

After the changes, timestamp/dates values of filters pushed down to parquet 
libs such as `FilterApi.eq(col1, -719162)` are rebased according the rebase 
settings. For the example, if the rebase mode is `CORRECTED`, **-719162** is 
pushed down as is but if the current rebase mode is `LEGACY`, the number of 
days is rebased to **-719164**. For more context, the PR description 
https://github.com/apache/spark/pull/28067 shows the diffs between two 
calendars.

### Why are the changes needed?
The changes fix the bug portrayed by the following example from SPARK-36034:
```scala
In [27]: 
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "LEGACY")
>>> spark.sql("SELECT DATE '0001-01-01' AS 
date").write.mode("overwrite").parquet("date_written_by_spark3_legacy")
>>> spark.read.parquet("date_written_by_spark3_legacy").where("date = 
'0001-01-01'").show()
++
|date|
++
++
```
The result must have the date value `0001-01-01`.

### Does this PR introduce _any_ user-facing change?
In some sense, yes. Query results can be different in some cases. For the 
example above:
```scala
scala> spark.conf.set("spark.sql.parquet.datetimeRebaseModeInWrite", 
"LEGACY")
scala> spark.sql("SELECT DATE '0001-01-01' AS 
date").write.mode("overwrite").parquet("date_written_by_spark3_legacy")
scala> spark.read.parquet("date_written_by_spark3_legacy").where("date = 
'0001-01-01'").show(false)
+--+
|date  |
+--+
|0001-01-01|
+--+
```

### How was this patch tested?
    By running the modified test suite `ParquetFilterSuite`:
```
$ build/sbt "test:testOnly *ParquetV1FilterSuite"
$ build/sbt "test:testOnly *ParquetV2FilterSuite"
```

Closes #33347 from MaxGekk/fix-parquet-ts-filter-pushdown.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit b09b7f7cc024d3054debd7bdb51caec3b53764d7)
Signed-off-by: Max Gekk 
---
 .../datasources/parquet/ParquetFileFormat.scala|  17 ++-
 .../datasources/parquet/ParquetFilters.scala   |  29 +++-
 .../v2/parquet/ParquetPartitionReaderFactory.scala |  17 ++-
 .../v2/parquet/ParquetScanBuilder.scala|  14 +-
 .../datasources/parquet/ParquetFilterSuite.scala   | 164 +++--
 5 files changed, 145 insertions(+), 96 deletions(-)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index 48e2e6e..ee229a3 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -266,11 +266,21 @@ class ParquetFileFormat
 
   lazy val footerFileMetaData =
 ParquetFooterReader.readFooter(sharedConf, filePath, 
SKIP_ROW_GROUPS).getFileMetaData
+  val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode(
+footerFileMetaData.getKeyValueMetaData.get,
+datetimeRebaseModeInRead)
   // Try to push down filters when filter push-down is enabled.
   val pushed = if (enableParquetFilterPushDown) {
 val parquetSchema = footerFileMetaData.getSchema
-val

[spark] branch master updated (8009f0d -> 71ea25d)

2021-07-17 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 8009f0d  [SPARK-35785][SS][FOLLOWUP] Remove ignored test from 
RocksDBSuite
 add 71ea25d  [SPARK-36170][SQL] Change quoted interval literal (interval 
constructor) to be converted to ANSI interval types

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/parser/AstBuilder.scala |  77 --
 .../spark/sql/errors/QueryParsingErrors.scala  |   2 +-
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  11 +-
 .../catalyst/parser/ExpressionParserSuite.scala|  42 +---
 .../test/resources/sql-tests/inputs/interval.sql   |   7 +-
 .../sql-tests/results/ansi/interval.sql.out|  90 ++---
 .../sql-tests/results/ansi/literals.sql.out|   4 +-
 .../resources/sql-tests/results/interval.sql.out   | 112 +
 .../resources/sql-tests/results/literals.sql.out   |   4 +-
 .../sql-tests/results/misc-functions.sql.out   |   4 +-
 .../org/apache/spark/sql/DateFunctionsSuite.scala  |  70 +++--
 11 files changed, 284 insertions(+), 139 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36170][SQL] Change quoted interval literal (interval constructor) to be converted to ANSI interval types

2021-07-17 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new f7ed6fc  [SPARK-36170][SQL] Change quoted interval literal (interval 
constructor) to be converted to ANSI interval types
f7ed6fc is described below

commit f7ed6fc6c608835ba33daff09edc02bb32fe6e97
Author: Kousuke Saruta 
AuthorDate: Sat Jul 17 12:23:37 2021 +0300

[SPARK-36170][SQL] Change quoted interval literal (interval constructor) to 
be converted to ANSI interval types

### What changes were proposed in this pull request?

This PR changes the behavior of the quoted interval literals like `SELECT 
INTERVAL '1 year 2 month'` to be converted to ANSI interval types.

### Why are the changes needed?

The tnit-to-unit interval literals and the unit list interval literals are 
converted to ANSI interval types but quoted interval literals are still 
converted to CalendarIntervalType.

```
-- Unit list interval literals
spark-sql> select interval 1 year 2 month;
1-2
-- Quoted interval literals
spark-sql> select interval '1 year 2 month';
1 years 2 months
```

### Does this PR introduce _any_ user-facing change?

Yes but the following sentence in `sql-migration-guide.md` seems to cover 
this change.
```
  - In Spark 3.2, the unit list interval literals can not mix year-month 
fields (YEAR and MONTH) and day-time fields (WEEK, DAY, ..., MICROSECOND).
For example, `INTERVAL 1 day 1 hour` is invalid in Spark 3.2. In Spark 3.1 
and earlier,
there is no such limitation and the literal returns value of 
`CalendarIntervalType`.
To restore the behavior before Spark 3.2, you can set 
`spark.sql.legacy.interval.enabled` to `true`.
```

### How was this patch tested?

Modified existing tests and add new tests.

Closes #33380 from sarutak/fix-interval-constructor.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit 71ea25d4f567c7ef598aa87b6e4a091a3119fead)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/parser/AstBuilder.scala |  77 --
 .../spark/sql/errors/QueryParsingErrors.scala  |   2 +-
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  11 +-
 .../catalyst/parser/ExpressionParserSuite.scala|  42 +---
 .../test/resources/sql-tests/inputs/interval.sql   |   7 +-
 .../sql-tests/results/ansi/interval.sql.out|  90 ++---
 .../sql-tests/results/ansi/literals.sql.out|   4 +-
 .../resources/sql-tests/results/interval.sql.out   | 112 +
 .../resources/sql-tests/results/literals.sql.out   |   4 +-
 .../sql-tests/results/misc-functions.sql.out   |   4 +-
 .../org/apache/spark/sql/DateFunctionsSuite.scala  |  70 +++--
 11 files changed, 284 insertions(+), 139 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 2624b5c..d213549 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2165,7 +2165,15 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   ex.setStackTrace(e.getStackTrace)
   throw ex
   }
-  Literal(interval, CalendarIntervalType)
+  if (!conf.legacyIntervalEnabled) {
+val units = value
+  .split("\\s")
+  .map(_.toLowerCase(Locale.ROOT).stripSuffix("s"))
+  .filter(s => s != "interval" && s.matches("[a-z]+"))
+constructMultiUnitsIntervalLiteral(ctx, interval, units)
+  } else {
+Literal(interval, CalendarIntervalType)
+  }
 case "X" =>
   val padding = if (value.length % 2 != 0) "0" else ""
   Literal(DatatypeConverter.parseHexBinary(padding + value))
@@ -2373,6 +2381,44 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with 
SQLConfHelper with Logg
   }
 
   /**
+   * Construct an [[Literal]] from [[CalendarInterval]] and
+   * units represented as a [[Seq]] of [[String]].
+   */
+  private def constructMultiUnitsIntervalLiteral(
+  ctx: ParserRuleContext,
+  calendarInterval: CalendarInterval,
+  units: Seq[String]): Literal = {
+val yearMonthFields = Set.empty[Byte]
+val dayTimeFields = Set.empty[Byte]
+for (unit <- units) {
+  if (YearMonthIntervalType.stringToField.contains(unit)) {
+yearMonthFields += YearMonthIntervalType.stringToField(unit)
+  } else if

[spark] branch master updated (d7df7a8 -> 42275bb)

2021-07-18 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from d7df7a8  [SPARK-36195][BUILD] Set MaxMetaspaceSize JVM option to 2g
 add 42275bb  [SPARK-36090][SQL] Support TimestampNTZType in expression 
Sequence

No new revisions were added by this update.

Summary of changes:
 .../expressions/collectionOperations.scala |  48 +---
 .../spark/sql/catalyst/util/DateTimeUtils.scala|  21 +++-
 .../expressions/CollectionExpressionsSuite.scala   | 122 -
 3 files changed, 172 insertions(+), 19 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36090][SQL] Support TimestampNTZType in expression Sequence

2021-07-18 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 85f70a1  [SPARK-36090][SQL] Support TimestampNTZType in expression 
Sequence
85f70a1 is described below

commit 85f70a1181b1b11417c197cee411e0ec9ced2373
Author: gengjiaan 
AuthorDate: Sun Jul 18 20:46:23 2021 +0300

[SPARK-36090][SQL] Support TimestampNTZType in expression Sequence

### What changes were proposed in this pull request?
The current implement of `Sequence` accept `TimestampType`, `DateType` and 
`IntegralType`. This PR will let `Sequence` accepts `TimestampNTZType`.

### Why are the changes needed?
We can generate sequence for timestamp without time zone.

### Does this PR introduce _any_ user-facing change?
'Yes'.
This PR will let `Sequence` accepts `TimestampNTZType`.

### How was this patch tested?
New tests.

Closes #33360 from beliefer/SPARK-36090.

Lead-authored-by: gengjiaan 
Co-authored-by: Jiaan Geng 
Signed-off-by: Max Gekk 
(cherry picked from commit 42275bb20d6849ee9df488d9ec1fa402f394ac89)
Signed-off-by: Max Gekk 
---
 .../expressions/collectionOperations.scala |  48 +---
 .../spark/sql/catalyst/util/DateTimeUtils.scala|  21 +++-
 .../expressions/CollectionExpressionsSuite.scala   | 122 -
 3 files changed, 172 insertions(+), 19 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 2883d8d..730b8d0 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -2568,7 +2568,7 @@ case class Sequence(
 val typesCorrect =
   startType.sameType(stop.dataType) &&
 (startType match {
-  case TimestampType =>
+  case TimestampType | TimestampNTZType =>
 stepOpt.isEmpty || CalendarIntervalType.acceptsType(stepType) ||
   YearMonthIntervalType.acceptsType(stepType) ||
   DayTimeIntervalType.acceptsType(stepType)
@@ -2614,20 +2614,20 @@ case class Sequence(
   val ct = ClassTag[T](iType.tag.mirror.runtimeClass(iType.tag.tpe))
   new IntegralSequenceImpl(iType)(ct, iType.integral)
 
-case TimestampType =>
+case TimestampType | TimestampNTZType =>
   if (stepOpt.isEmpty || 
CalendarIntervalType.acceptsType(stepOpt.get.dataType)) {
-new TemporalSequenceImpl[Long](LongType, 1, identity, zoneId)
+new TemporalSequenceImpl[Long](LongType, start.dataType, 1, identity, 
zoneId)
   } else if (YearMonthIntervalType.acceptsType(stepOpt.get.dataType)) {
-new PeriodSequenceImpl[Long](LongType, 1, identity, zoneId)
+new PeriodSequenceImpl[Long](LongType, start.dataType, 1, identity, 
zoneId)
   } else {
-new DurationSequenceImpl[Long](LongType, 1, identity, zoneId)
+new DurationSequenceImpl[Long](LongType, start.dataType, 1, identity, 
zoneId)
   }
 
 case DateType =>
   if (stepOpt.isEmpty || 
CalendarIntervalType.acceptsType(stepOpt.get.dataType)) {
-new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, 
zoneId)
+new TemporalSequenceImpl[Int](IntegerType, start.dataType, 
MICROS_PER_DAY, _.toInt, zoneId)
   } else {
-new PeriodSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, 
zoneId)
+new PeriodSequenceImpl[Int](IntegerType, start.dataType, 
MICROS_PER_DAY, _.toInt, zoneId)
   }
   }
 
@@ -2769,8 +2769,9 @@ object Sequence {
   }
 
   private class PeriodSequenceImpl[T: ClassTag]
-  (dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId)
-  (implicit num: Integral[T]) extends InternalSequenceBase(dt, scale, 
fromLong, zoneId) {
+  (dt: IntegralType, outerDataType: DataType, scale: Long, fromLong: Long 
=> T, zoneId: ZoneId)
+  (implicit num: Integral[T])
+extends InternalSequenceBase(dt, outerDataType, scale, fromLong, zoneId) {
 
 override val defaultStep: DefaultStep = new DefaultStep(
   (dt.ordering.lteq _).asInstanceOf[LessThanOrEqualFn],
@@ -2794,8 +2795,9 @@ object Sequence {
   }
 
   private class DurationSequenceImpl[T: ClassTag]
-  (dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId)
-  (implicit num: Integral[T]) extends InternalSequenceBase(dt, scale, 
fromLong, zoneId) {
+  (dt: IntegralType, outerDataType: DataType, scale: Long, fromLong: Long 
=> T, zoneId: ZoneId)
+  (implicit num: Integral[T])
+extends InternalSequenceBase(dt, outerDataType, s

[spark] branch master updated (bf680bf -> c0d84e6)

2021-07-20 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from bf680bf  [SPARK-36210][SQL] Preserve column insertion order in 
Dataset.withColumns
 add c0d84e6  [SPARK-36222][SQL] Step by days in the Sequence expression 
for dates

No new revisions were added by this update.

Summary of changes:
 .../expressions/collectionOperations.scala | 17 --
 .../expressions/CollectionExpressionsSuite.scala   | 61 --
 2 files changed, 68 insertions(+), 10 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36222][SQL] Step by days in the Sequence expression for dates

2021-07-20 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 9a7c59c  [SPARK-36222][SQL] Step by days in the Sequence expression 
for dates
9a7c59c is described below

commit 9a7c59c99ce69411485acf382dfc9be053927b59
Author: gengjiaan 
AuthorDate: Tue Jul 20 19:16:56 2021 +0300

[SPARK-36222][SQL] Step by days in the Sequence expression for dates

### What changes were proposed in this pull request?
The current implement of `Sequence` expression not support step by days for 
dates.
```
spark-sql> select sequence(date'2021-07-01', date'2021-07-10', interval '3' 
day);
Error in query: cannot resolve 'sequence(DATE '2021-07-01', DATE 
'2021-07-10', INTERVAL '3' DAY)' due to data type mismatch:
sequence uses the wrong parameter type. The parameter type must conform to:
1. The start and stop expressions must resolve to the same type.
2. If start and stop expressions resolve to the 'date' or 'timestamp' type
then the step expression must resolve to the 'interval' or
'interval year to month' or 'interval day to second' type,
otherwise to the same type as the start and stop expressions.
 ; line 1 pos 7;
'Project [unresolvedalias(sequence(2021-07-01, 2021-07-10, Some(INTERVAL 
'3' DAY), Some(Europe/Moscow)), None)]
+- OneRowRelation
```

### Why are the changes needed?
`DayTimeInterval` has day granularity should as step for dates.

### Does this PR introduce _any_ user-facing change?
'Yes'.
Sequence expression will supports step by `DayTimeInterval` has day 
granularity for dates.

### How was this patch tested?
New tests.

Closes #33439 from beliefer/SPARK-36222.

Authored-by: gengjiaan 
Signed-off-by: Max Gekk 
(cherry picked from commit c0d84e6cf1046b7944796038414ef21fe9c7e3b5)
Signed-off-by: Max Gekk 
---
 .../expressions/collectionOperations.scala | 17 --
 .../expressions/CollectionExpressionsSuite.scala   | 61 --
 2 files changed, 68 insertions(+), 10 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index 730b8d0..2c3312a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -2574,7 +2574,8 @@ case class Sequence(
   DayTimeIntervalType.acceptsType(stepType)
   case DateType =>
 stepOpt.isEmpty || CalendarIntervalType.acceptsType(stepType) ||
-  YearMonthIntervalType.acceptsType(stepType)
+  YearMonthIntervalType.acceptsType(stepType) ||
+  DayTimeIntervalType.acceptsType(stepType)
   case _: IntegralType =>
 stepOpt.isEmpty || stepType.sameType(startType)
   case _ => false
@@ -2626,8 +2627,10 @@ case class Sequence(
 case DateType =>
   if (stepOpt.isEmpty || 
CalendarIntervalType.acceptsType(stepOpt.get.dataType)) {
 new TemporalSequenceImpl[Int](IntegerType, start.dataType, 
MICROS_PER_DAY, _.toInt, zoneId)
-  } else {
+  } else if (YearMonthIntervalType.acceptsType(stepOpt.get.dataType)) {
 new PeriodSequenceImpl[Int](IntegerType, start.dataType, 
MICROS_PER_DAY, _.toInt, zoneId)
+  } else {
+new DurationSequenceImpl[Int](IntegerType, start.dataType, 
MICROS_PER_DAY, _.toInt, zoneId)
   }
   }
 
@@ -2807,15 +2810,19 @@ object Sequence {
 val intervalType: DataType = DayTimeIntervalType()
 
 def splitStep(input: Any): (Int, Int, Long) = {
-  (0, 0, input.asInstanceOf[Long])
+  val duration = input.asInstanceOf[Long]
+  val days = IntervalUtils.getDays(duration)
+  val micros = duration - days * MICROS_PER_DAY
+  (0, days, micros)
 }
 
 def stepSplitCode(
 stepMonths: String, stepDays: String, stepMicros: String, step: 
String): String = {
   s"""
  |final int $stepMonths = 0;
- |final int $stepDays = 0;
- |final long $stepMicros = $step;
+ |final int $stepDays =
+ |  (int) 
org.apache.spark.sql.catalyst.util.IntervalUtils.getDays($step);
+ |final long $stepMicros = $step - $stepDays * ${MICROS_PER_DAY}L;
""".stripMargin
 }
   }
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala
 
b/sql/catalyst/src/te

[spark] branch master updated (94aece4 -> f56c7b7)

2021-07-21 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 94aece4  [SPARK-36020][SQL][FOLLOWUP] RemoveRedundantProjects should 
retain the LOGICAL_PLAN_TAG tag
 add f56c7b7  [SPARK-36208][SQL] SparkScriptTransformation should support 
ANSI interval types

No new revisions were added by this update.

Summary of changes:
 .../execution/BaseScriptTransformationExec.scala|  8 
 .../execution/BaseScriptTransformationSuite.scala   | 21 +
 2 files changed, 29 insertions(+)

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



[spark] branch branch-3.2 updated: [SPARK-36208][SQL][3.2] SparkScriptTransformation should support ANSI interval types

2021-07-21 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 468165a  [SPARK-36208][SQL][3.2] SparkScriptTransformation should 
support ANSI interval types
468165a is described below

commit 468165ae52aa788e3fa59f385225b90c616bfa0f
Author: Kousuke Saruta 
AuthorDate: Wed Jul 21 20:54:18 2021 +0300

[SPARK-36208][SQL][3.2] SparkScriptTransformation should support ANSI 
interval types

### What changes were proposed in this pull request?

This PR changes `BaseScriptTransformationExec` for 
`SparkScriptTransformationExec` to support ANSI interval types.

### Why are the changes needed?

`SparkScriptTransformationExec` support `CalendarIntervalType` so it's 
better to support ANSI interval types as well.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New test.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit f56c7b71ff27e6f5379f3699c2dcb5f79a0ae791)
Signed-off-by: Max Gekk 

Closes #33463 from MaxGekk/sarutak_script-transformation-interval-3.2.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
---
 .../execution/BaseScriptTransformationExec.scala|  8 
 .../execution/BaseScriptTransformationSuite.scala   | 21 +
 2 files changed, 29 insertions(+)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
index 7835981..e249cd6 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
@@ -223,6 +223,14 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
   case CalendarIntervalType => wrapperConvertException(
 data => IntervalUtils.stringToInterval(UTF8String.fromString(data)),
 converter)
+  case YearMonthIntervalType(start, end) => wrapperConvertException(
+data => IntervalUtils.monthsToPeriod(
+  IntervalUtils.castStringToYMInterval(UTF8String.fromString(data), 
start, end)),
+converter)
+  case DayTimeIntervalType(start, end) => wrapperConvertException(
+data => IntervalUtils.microsToDuration(
+  IntervalUtils.castStringToDTInterval(UTF8String.fromString(data), 
start, end)),
+converter)
   case _: ArrayType | _: MapType | _: StructType =>
 val complexTypeFactory = JsonToStructs(attr.dataType,
   ioschema.outputSerdeProps.toMap, Literal(null), 
Some(conf.sessionLocalTimeZone))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
index c845dd81..9d8fcda 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
@@ -633,6 +633,27 @@ abstract class BaseScriptTransformationSuite extends 
SparkPlanTest with SQLTestU
   }
 }
   }
+
+  test("SPARK-36208: TRANSFORM should support ANSI interval (no serde)") {
+assume(TestUtils.testCommandAvailable("python"))
+withTempView("v") {
+  val df = Seq(
+(Period.of(1, 2, 0), 
Duration.ofDays(1).plusHours(2).plusMinutes(3).plusSeconds(4))
+  ).toDF("ym", "dt")
+
+  checkAnswer(
+df,
+(child: SparkPlan) => createScriptTransformationExec(
+  script = "cat",
+  output = Seq(
+AttributeReference("ym", YearMonthIntervalType())(),
+AttributeReference("dt", DayTimeIntervalType())()),
+  child = child,
+  ioschema = defaultIOSchema
+),
+df.select($"ym", $"dt").collect())
+}
+  }
 }
 
 case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExecNode {

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



[spark] branch master updated (3a1db2d -> 07fa38e)

2021-07-22 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 3a1db2d  [SPARK-36209][PYTHON][DOCS] Fix link to pyspark Dataframe 
documentation
 add 07fa38e  [SPARK-35815][SQL] Allow delayThreshold for watermark to be 
represented as ANSI interval literals

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/errors/QueryCompilationErrors.scala  |  2 +-
 .../main/scala/org/apache/spark/sql/Dataset.scala  | 23 ++---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 54 ++
 3 files changed, 71 insertions(+), 8 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-35815][SQL] Allow delayThreshold for watermark to be represented as ANSI interval literals

2021-07-22 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 3ee9a0d  [SPARK-35815][SQL] Allow delayThreshold for watermark to be 
represented as ANSI interval literals
3ee9a0d is described below

commit 3ee9a0db3a3eb8e88bbab28207c91bd4637b313a
Author: Kousuke Saruta 
AuthorDate: Thu Jul 22 17:36:22 2021 +0300

[SPARK-35815][SQL] Allow delayThreshold for watermark to be represented as 
ANSI interval literals

### What changes were proposed in this pull request?

This PR extends the way to represent `delayThreshold` with ANSI interval 
literals for watermark.

### Why are the changes needed?

A `delayThreshold` is semantically an interval value so it's should be 
represented as ANSI interval literals as well as the conventional `1 second` 
form.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New tests.

Closes #33456 from sarutak/delayThreshold-interval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit 07fa38e2c1082c2b69b3bf9489cee4dfe4db2c26)
Signed-off-by: Max Gekk 
---
 .../spark/sql/errors/QueryCompilationErrors.scala  |  2 +-
 .../main/scala/org/apache/spark/sql/Dataset.scala  | 23 ++---
 .../sql/streaming/EventTimeWatermarkSuite.scala| 54 ++
 3 files changed, 71 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 7a33d52a..f576036 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -2265,7 +2265,7 @@ private[spark] object QueryCompilationErrors {
   s"""Cannot resolve column name "$colName" among 
(${fieldsStr})${extraMsg}""")
   }
 
-  def cannotParseTimeDelayError(delayThreshold: String, e: 
IllegalArgumentException): Throwable = {
+  def cannotParseTimeDelayError(delayThreshold: String, e: Throwable): 
Throwable = {
 new AnalysisException(s"Unable to parse time delay '$delayThreshold'", 
cause = Some(e))
   }
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 0fd10c1..3abc060 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.io.{ByteArrayOutputStream, CharArrayWriter, DataOutputStream}
+import java.util.Locale
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.{ArrayBuffer, HashSet}
@@ -42,7 +43,7 @@ import org.apache.spark.sql.catalyst.encoders._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions}
 import org.apache.spark.sql.catalyst.optimizer.CombineUnions
-import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils}
+import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, 
ParseException, ParserUtils}
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, 
PartitioningCollection}
@@ -63,7 +64,7 @@ import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.SchemaUtils
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.unsafe.array.ByteArrayMethods
-import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 import org.apache.spark.util.Utils
 
 private[sql] object Dataset {
@@ -739,13 +740,21 @@ class Dataset[T] private[sql](
   // We only accept an existing column name, not a derived column here as a 
watermark that is
   // defined on a derived column cannot referenced elsewhere in the plan.
   def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = 
withTypedPlan {
-val parsedDelay =
-  try {
+val parsedDelay = try {
+  if (delayThreshold.toLowerCase(Locale.ROOT).trim.startsWith("interval")) 
{
+CatalystSqlParser.parseExpression(delayThreshold) match {
+  case Literal(months: Int, _: YearMonthIntervalType) =>
+new CalendarInterval(months, 0, 0)
+  case Literal(micros: Long, _: DayTimeIntervalType) =>
+new CalendarInterval(0, 0, micros)
+}
+  } else {
 IntervalUtils.stringToInterval(UTF8String.fromString(delayThreshold

[spark] branch branch-3.2 updated (3ee9a0d -> c5697d0)

2021-07-22 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 3ee9a0d  [SPARK-35815][SQL] Allow delayThreshold for watermark to be 
represented as ANSI interval literals
 add c5697d0  [SPARK-36257][SQL][3.2] Updated the version of TimestampNTZ 
related changes as 3.3.0

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/expressions/datetimeExpressions.scala   | 10 +-
 .../scala/org/apache/spark/sql/types/TimestampNTZType.scala|  4 ++--
 sql/core/src/main/scala/org/apache/spark/sql/functions.scala   |  2 +-
 3 files changed, 8 insertions(+), 8 deletions(-)

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



[spark] branch branch-3.2 updated (dcd37f9 -> 3d86128)

2021-07-27 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git.


from dcd37f9  Revert "[SPARK-36136][SQL][TESTS] Refactor 
PruneFileSourcePartitionsSuite etc to a different package"
 add 3d86128  [SPARK-34619][SQL][DOCS][3.2] Describe ANSI interval types at 
the Data types page of the SQL reference

No new revisions were added by this update.

Summary of changes:
 docs/sql-ref-datatypes.md | 38 ++
 1 file changed, 38 insertions(+)

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



[spark] branch master updated: [SPARK-36323][SQL] Support ANSI interval literals for TimeWindow

2021-07-28 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new db18866  [SPARK-36323][SQL] Support ANSI interval literals for 
TimeWindow
db18866 is described below

commit db18866742a4641e7119f637024bc89a3f048634
Author: Kousuke Saruta 
AuthorDate: Thu Jul 29 08:51:51 2021 +0300

[SPARK-36323][SQL] Support ANSI interval literals for TimeWindow

### What changes were proposed in this pull request?

This PR proposes to support ANSI interval literals for `TimeWindow`.

### Why are the changes needed?

Watermark also supports ANSI interval literals so it's great to support for 
`TimeWindow`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New test.

Closes #33551 from sarutak/window-interval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/expressions/TimeWindow.scala  |  5 +-
 .../spark/sql/catalyst/util/IntervalUtils.scala| 23 +++-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  4 +-
 .../sql/catalyst/expressions/TimeWindowSuite.scala | 62 +++---
 .../main/scala/org/apache/spark/sql/Dataset.scala  | 20 +--
 5 files changed, 82 insertions(+), 32 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
index 8475c1f..2f08fd7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
@@ -26,7 +26,6 @@ import 
org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY
 import org.apache.spark.sql.catalyst.util.IntervalUtils
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
 
 case class TimeWindow(
 timeColumn: Expression,
@@ -110,12 +109,12 @@ object TimeWindow {
* precision.
*/
   def getIntervalInMicroSeconds(interval: String): Long = {
-val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval))
+val cal = IntervalUtils.fromIntervalString(interval)
 if (cal.months != 0) {
   throw new IllegalArgumentException(
 s"Intervals greater than a month is not supported ($interval).")
 }
-cal.days * MICROS_PER_DAY + cal.microseconds
+Math.addExact(Math.multiplyExact(cal.days, MICROS_PER_DAY), 
cal.microseconds)
   }
 
   /**
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 dc6c02e..62a2657 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,11 +25,14 @@ import java.util.concurrent.TimeUnit
 import scala.collection.mutable
 import scala.util.control.NonFatal
 
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
 import org.apache.spark.sql.catalyst.util.DateTimeConstants._
 import org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
-import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, 
YearMonthIntervalType => YM}
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
@@ -433,6 +436,24 @@ object IntervalUtils {
 }
   }
 
+  /**
+   * Parse all kinds of interval literals including unit-to-unit form and unit 
list form
+   */
+  def fromIntervalString(input: String): CalendarInterval = try {
+if (input.toLowerCase(Locale.ROOT).trim.startsWith("interval")) {
+  CatalystSqlParser.parseExpression(input) match {
+case Literal(months: Int, _: YearMonthIntervalType) => new 
CalendarInterval(months, 0, 0)
+case Literal(micros: Long, _: DayTimeIntervalType) => new 
CalendarInterval(0, 0, micros)
+case Literal(cal: CalendarInterval, CalendarIntervalType) => cal
+  }
+} else {
+  stringToInterval(UTF8String.fromString(input))
+}
+  } catch {
+case NonFatal(e) =>
+  throw QueryCompilationErrors.cannotParseIntervalError(input, e)
+  }
+
   private val dayTimePatternLegacy =
   

[spark] branch branch-3.2 updated: [SPARK-36323][SQL] Support ANSI interval literals for TimeWindow

2021-07-28 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new d247a6cd [SPARK-36323][SQL] Support ANSI interval literals for 
TimeWindow
d247a6cd is described below

commit d247a6cd1a2240fcfb45f288cfb6cc6236368c5e
Author: Kousuke Saruta 
AuthorDate: Thu Jul 29 08:51:51 2021 +0300

[SPARK-36323][SQL] Support ANSI interval literals for TimeWindow

### What changes were proposed in this pull request?

This PR proposes to support ANSI interval literals for `TimeWindow`.

### Why are the changes needed?

Watermark also supports ANSI interval literals so it's great to support for 
`TimeWindow`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New test.

Closes #33551 from sarutak/window-interval.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
(cherry picked from commit db18866742a4641e7119f637024bc89a3f048634)
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/expressions/TimeWindow.scala  |  5 +-
 .../spark/sql/catalyst/util/IntervalUtils.scala| 23 +++-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  4 +-
 .../sql/catalyst/expressions/TimeWindowSuite.scala | 62 +++---
 .../main/scala/org/apache/spark/sql/Dataset.scala  | 20 +--
 5 files changed, 82 insertions(+), 32 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
index 8475c1f..2f08fd7 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala
@@ -26,7 +26,6 @@ import 
org.apache.spark.sql.catalyst.util.DateTimeConstants.MICROS_PER_DAY
 import org.apache.spark.sql.catalyst.util.IntervalUtils
 import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
 
 case class TimeWindow(
 timeColumn: Expression,
@@ -110,12 +109,12 @@ object TimeWindow {
* precision.
*/
   def getIntervalInMicroSeconds(interval: String): Long = {
-val cal = IntervalUtils.stringToInterval(UTF8String.fromString(interval))
+val cal = IntervalUtils.fromIntervalString(interval)
 if (cal.months != 0) {
   throw new IllegalArgumentException(
 s"Intervals greater than a month is not supported ($interval).")
 }
-cal.days * MICROS_PER_DAY + cal.microseconds
+Math.addExact(Math.multiplyExact(cal.days, MICROS_PER_DAY), 
cal.microseconds)
   }
 
   /**
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 dc6c02e..62a2657 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,11 +25,14 @@ import java.util.concurrent.TimeUnit
 import scala.collection.mutable
 import scala.util.control.NonFatal
 
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
 import org.apache.spark.sql.catalyst.util.DateTimeConstants._
 import org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToMicros
 import org.apache.spark.sql.catalyst.util.IntervalStringStyles.{ANSI_STYLE, 
HIVE_STYLE, IntervalStyle}
-import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
 import org.apache.spark.sql.types.{DayTimeIntervalType => DT, Decimal, 
YearMonthIntervalType => YM}
 import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
@@ -433,6 +436,24 @@ object IntervalUtils {
 }
   }
 
+  /**
+   * Parse all kinds of interval literals including unit-to-unit form and unit 
list form
+   */
+  def fromIntervalString(input: String): CalendarInterval = try {
+if (input.toLowerCase(Locale.ROOT).trim.startsWith("interval")) {
+  CatalystSqlParser.parseExpression(input) match {
+case Literal(months: Int, _: YearMonthIntervalType) => new 
CalendarInterval(months, 0, 0)
+case Literal(micros: Long, _: DayTimeIntervalType) => new 
CalendarInterval(0, 0, micros)
+case Literal(cal: CalendarInterval, CalendarIntervalType) => cal
+  }
+} else {
+  stringToInterval(UTF8String.fromString(input))
+}
+  } catch {
+case NonFatal(e) =>
+  throw Quer

[spark] branch master updated (8ca11fe -> 7cb9c1c)

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 8ca11fe  [SPARK-36192][PYTHON] Better error messages for DataTypeOps 
against lists
 add 7cb9c1c  [SPARK-36380][SQL] Simplify the logical plan names for ALTER 
TABLE ... COLUMN

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/analysis/Analyzer.scala |  12 +-
 .../sql/catalyst/analysis/CheckAnalysis.scala  |  12 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |  12 +-
 .../plans/logical/v2AlterTableCommands.scala   | 230 +
 .../sql/catalyst/plans/logical/v2Commands.scala| 193 +
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  54 ++---
 .../catalyst/analysis/ResolveSessionCatalog.scala  |  10 +-
 .../datasources/v2/DataSourceV2Strategy.scala  |  26 +--
 .../connector/V2CommandsCaseSensitivitySuite.scala |  26 +--
 .../execution/command/PlanResolutionSuite.scala|  12 +-
 10 files changed, 303 insertions(+), 284 deletions(-)
 create mode 100644 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2AlterTableCommands.scala

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



[spark] branch branch-3.2 updated: [SPARK-36380][SQL] Simplify the logical plan names for ALTER TABLE ... COLUMN

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 7c58684  [SPARK-36380][SQL] Simplify the logical plan names for ALTER 
TABLE ... COLUMN
7c58684 is described below

commit 7c586842d71064169aa77baf666a8566d9ed785e
Author: Wenchen Fan 
AuthorDate: Tue Aug 3 10:43:00 2021 +0300

[SPARK-36380][SQL] Simplify the logical plan names for ALTER TABLE ... 
COLUMN

### What changes were proposed in this pull request?

This a followup of the recent work such as 
https://github.com/apache/spark/pull/33200

For `ALTER TABLE` commands, the logical plans do not have the common 
`AlterTable` prefix in the name and just use names like `SetTableLocation`. 
This PR proposes to follow the same naming rule in `ALTER TABE ... COLUMN` 
commands.

This PR also moves these AlterTable commands to a individual file and give 
them a base trait.

### Why are the changes needed?

name simplification

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

existing test

Closes #33609 from cloud-fan/dsv2.

Authored-by: Wenchen Fan 
Signed-off-by: Max Gekk 
(cherry picked from commit 7cb9c1c2415a0984515e4d4733f816673e4ae3c8)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/analysis/Analyzer.scala |  12 +-
 .../sql/catalyst/analysis/CheckAnalysis.scala  |  12 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala |  12 +-
 .../plans/logical/v2AlterTableCommands.scala   | 230 +
 .../sql/catalyst/plans/logical/v2Commands.scala| 193 +
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  54 ++---
 .../catalyst/analysis/ResolveSessionCatalog.scala  |  10 +-
 .../datasources/v2/DataSourceV2Strategy.scala  |  26 +--
 .../connector/V2CommandsCaseSensitivitySuite.scala |  26 +--
 .../execution/command/PlanResolutionSuite.scala|  12 +-
 10 files changed, 303 insertions(+), 284 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index a9c085a..75fad11a 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -269,7 +269,7 @@ class Analyzer(override val catalogManager: CatalogManager)
   ResolveRelations ::
   ResolveTables ::
   ResolvePartitionSpec ::
-  ResolveAlterTableColumnCommands ::
+  ResolveAlterTableCommands ::
   AddMetadataColumns ::
   DeduplicateRelations ::
   ResolveReferences ::
@@ -3607,15 +3607,15 @@ class Analyzer(override val catalogManager: 
CatalogManager)
* Rule to mostly resolve, normalize and rewrite column names based on case 
sensitivity
* for alter table column commands.
*/
-  object ResolveAlterTableColumnCommands extends Rule[LogicalPlan] {
+  object ResolveAlterTableCommands extends Rule[LogicalPlan] {
 def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
-  case a: AlterTableColumnCommand if a.table.resolved && 
hasUnresolvedFieldName(a) =>
+  case a: AlterTableCommand if a.table.resolved && 
hasUnresolvedFieldName(a) =>
 val table = a.table.asInstanceOf[ResolvedTable]
 a.transformExpressions {
   case u: UnresolvedFieldName => resolveFieldNames(table, u.name, u)
 }
 
-  case a @ AlterTableAddColumns(r: ResolvedTable, cols) if !a.resolved =>
+  case a @ AddColumns(r: ResolvedTable, cols) if !a.resolved =>
 // 'colsToAdd' keeps track of new columns being added. It stores a 
mapping from a
 // normalized parent name of fields to field names that belong to the 
parent.
 // For example, if we add columns "a.b.c", "a.b.d", and "a.c", 
'colsToAdd' will become
@@ -3668,7 +3668,7 @@ class Analyzer(override val catalogManager: 
CatalogManager)
 resolved.copyTagsFrom(a)
 resolved
 
-  case a @ AlterTableAlterColumn(
+  case a @ AlterColumn(
   table: ResolvedTable, ResolvedFieldName(path, field), dataType, _, 
_, position) =>
 val newDataType = dataType.flatMap { dt =>
   // Hive style syntax provides the column type, even if it may not 
have changed.
@@ -3705,7 +3705,7 @@ class Analyzer(override val catalogManager: 
CatalogManager)
   }.getOrElse(throw QueryCompilationErrors.missingFieldError(fieldName, 
table, context.origin))
 }
 
-private def hasUnresolvedFieldName(a: AlterTableColumnCommand): Boolean = {
+private def hasUnresolvedFieldNam

[spark] branch master updated (dd80457 -> 92cdb17)

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from dd80457  [SPARK-36315][SQL] Only skip AQEShuffleReadRule in the final 
stage if it breaks the distribution requirement
 add 92cdb17  [SPARK-35815][SQL][FOLLOWUP] Add test considering the case 
spark.sql.legacy.interval.enabled is true

No new revisions were added by this update.

Summary of changes:
 .../sql/streaming/EventTimeWatermarkSuite.scala| 105 +++--
 1 file changed, 56 insertions(+), 49 deletions(-)

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



[spark] branch branch-3.2 updated (8d817dc -> cc75618)

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 8d817dc  [SPARK-36315][SQL] Only skip AQEShuffleReadRule in the final 
stage if it breaks the distribution requirement
 add cc75618  [SPARK-35815][SQL][FOLLOWUP] Add test considering the case 
spark.sql.legacy.interval.enabled is true

No new revisions were added by this update.

Summary of changes:
 .../sql/streaming/EventTimeWatermarkSuite.scala| 105 +++--
 1 file changed, 56 insertions(+), 49 deletions(-)

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



[spark] branch master updated: [SPARK-36349][SQL] Disallow ANSI intervals in file-based datasources

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 67cbc93  [SPARK-36349][SQL] Disallow ANSI intervals in file-based 
datasources
67cbc93 is described below

commit 67cbc932638179925ebbeb76d6d6e6f25a3cb2e2
Author: Max Gekk 
AuthorDate: Tue Aug 3 20:30:20 2021 +0300

[SPARK-36349][SQL] Disallow ANSI intervals in file-based datasources

### What changes were proposed in this pull request?
In the PR, I propose to ban `YearMonthIntervalType` and 
`DayTimeIntervalType` at the analysis phase while creating a table using a 
built-in filed-based datasource or writing a dataset to such datasource. In 
particular, add the following case:
```scala
case _: DayTimeIntervalType | _: YearMonthIntervalType => false
```
to all methods that override either:
- V2 `FileTable.supportsDataType()`
- V1 `FileFormat.supportDataType()`

### Why are the changes needed?
To improve user experience with Spark SQL, and output a proper error 
message at the analysis phase.

### Does this PR introduce _any_ user-facing change?
Yes but ANSI interval types haven't released yet. So, for users this is new 
behavior.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 "test:testOnly *HiveOrcSourceSuite"
```

Closes #33580 from MaxGekk/interval-ban-in-ds.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/avro/AvroUtils.scala  |  2 ++
 .../execution/datasources/csv/CSVFileFormat.scala  |  2 ++
 .../datasources/json/JsonFileFormat.scala  |  2 ++
 .../execution/datasources/orc/OrcFileFormat.scala  |  2 ++
 .../datasources/parquet/ParquetFileFormat.scala|  2 ++
 .../execution/datasources/v2/csv/CSVTable.scala|  4 +++-
 .../execution/datasources/v2/json/JsonTable.scala  |  2 ++
 .../execution/datasources/v2/orc/OrcTable.scala|  2 ++
 .../datasources/v2/parquet/ParquetTable.scala  |  2 ++
 .../datasources/CommonFileDataSourceSuite.scala| 25 +-
 .../apache/spark/sql/hive/orc/OrcFileFormat.scala  |  2 ++
 11 files changed, 45 insertions(+), 2 deletions(-)

diff --git 
a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala 
b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index 149d0b6..328927f 100644
--- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -71,6 +71,8 @@ private[sql] object AvroUtils extends Logging {
   }
 
   def supportsDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case st: StructType => st.forall { f => supportsDataType(f.dataType) }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
index d40ad9d..c3a8a95 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
@@ -148,6 +148,8 @@ class CSVFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
   override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat]
 
   override def supportDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case udt: UserDefinedType[_] => supportDataType(udt.sqlType)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
index 9c6c77a..7ffeba4e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
@@ -134,6 +134,8 @@ class JsonFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
   override def equals(other: Any): Boolean = other.isInstanceOf[JsonFileFormat]
 
   override def supportDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case st: StructType => st.forall { f => supportDataType(f.dataType) }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasou

[spark] branch branch-3.2 updated: [SPARK-36349][SQL] Disallow ANSI intervals in file-based datasources

2021-08-03 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new bd33408  [SPARK-36349][SQL] Disallow ANSI intervals in file-based 
datasources
bd33408 is described below

commit bd33408b4b5aefc5b83ab1355bb0c1faacad190c
Author: Max Gekk 
AuthorDate: Tue Aug 3 20:30:20 2021 +0300

[SPARK-36349][SQL] Disallow ANSI intervals in file-based datasources

### What changes were proposed in this pull request?
In the PR, I propose to ban `YearMonthIntervalType` and 
`DayTimeIntervalType` at the analysis phase while creating a table using a 
built-in filed-based datasource or writing a dataset to such datasource. In 
particular, add the following case:
```scala
case _: DayTimeIntervalType | _: YearMonthIntervalType => false
```
to all methods that override either:
- V2 `FileTable.supportsDataType()`
- V1 `FileFormat.supportDataType()`

### Why are the changes needed?
To improve user experience with Spark SQL, and output a proper error 
message at the analysis phase.

### Does this PR introduce _any_ user-facing change?
Yes but ANSI interval types haven't released yet. So, for users this is new 
behavior.

### How was this patch tested?
By running the affected test suites:
```
$ build/sbt -Phive-2.3 "test:testOnly *HiveOrcSourceSuite"
```

Closes #33580 from MaxGekk/interval-ban-in-ds.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit 67cbc932638179925ebbeb76d6d6e6f25a3cb2e2)
Signed-off-by: Max Gekk 
---
 .../org/apache/spark/sql/avro/AvroUtils.scala  |  2 ++
 .../execution/datasources/csv/CSVFileFormat.scala  |  2 ++
 .../datasources/json/JsonFileFormat.scala  |  2 ++
 .../execution/datasources/orc/OrcFileFormat.scala  |  2 ++
 .../datasources/parquet/ParquetFileFormat.scala|  2 ++
 .../execution/datasources/v2/csv/CSVTable.scala|  4 +++-
 .../execution/datasources/v2/json/JsonTable.scala  |  2 ++
 .../execution/datasources/v2/orc/OrcTable.scala|  2 ++
 .../datasources/v2/parquet/ParquetTable.scala  |  2 ++
 .../datasources/CommonFileDataSourceSuite.scala| 25 +-
 .../apache/spark/sql/hive/orc/OrcFileFormat.scala  |  2 ++
 11 files changed, 45 insertions(+), 2 deletions(-)

diff --git 
a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala 
b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index f09af74..68b393e 100644
--- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -71,6 +71,8 @@ private[sql] object AvroUtils extends Logging {
   }
 
   def supportsDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case st: StructType => st.forall { f => supportsDataType(f.dataType) }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
index d40ad9d..c3a8a95 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
@@ -148,6 +148,8 @@ class CSVFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
   override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat]
 
   override def supportDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case udt: UserDefinedType[_] => supportDataType(udt.sqlType)
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
index 9c6c77a..7ffeba4e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
@@ -134,6 +134,8 @@ class JsonFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
   override def equals(other: Any): Boolean = other.isInstanceOf[JsonFileFormat]
 
   override def supportDataType(dataType: DataType): Boolean = dataType match {
+case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+
 case _: AtomicType => true
 
 case st: StructType => st.forall { f => supportDataType(f.dataType) }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/ex

[spark] branch master updated (7f56b73c -> 89d8a4e)

2021-08-10 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 7f56b73c [SPARK-36466][SQL] Table in unloaded catalog referenced by 
view should load correctly
 add 89d8a4e  [SPARK-36431][SQL] Support TypeCoercion of ANSI intervals 
with different fields

No new revisions were added by this update.

Summary of changes:
 .../sql/catalyst/analysis/AnsiTypeCoercion.scala   |  5 ++
 .../spark/sql/catalyst/analysis/TypeCoercion.scala |  5 ++
 .../sql/catalyst/analysis/TypeCoercionSuite.scala  | 47 +++
 .../test/resources/sql-tests/inputs/interval.sql   | 13 +++
 .../sql-tests/results/ansi/interval.sql.out| 98 +-
 .../resources/sql-tests/results/interval.sql.out   | 98 +-
 6 files changed, 258 insertions(+), 8 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36431][SQL] Support TypeCoercion of ANSI intervals with different fields

2021-08-10 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new fb6f379  [SPARK-36431][SQL] Support TypeCoercion of ANSI intervals 
with different fields
fb6f379 is described below

commit fb6f3792afacead3299e86cc94f3f1460b3a4ba1
Author: Angerszh 
AuthorDate: Tue Aug 10 14:22:31 2021 +0300

[SPARK-36431][SQL] Support TypeCoercion of ANSI intervals with different 
fields

### What changes were proposed in this pull request?
 Support TypeCoercion of ANSI intervals with different fields

### Why are the changes needed?
 Support TypeCoercion of ANSI intervals with different fields

### Does this PR introduce _any_ user-facing change?
After this pr user can
 - use comparison function with  different fields of 
DayTimeIntervalType/YearMonthIntervalType such as `INTERVAL '1' YEAR` > 
`INTERVAL '11' MONTH`
 - support different field of ansi interval type in collection function 
such as `array(INTERVAL '1' YEAR, INTERVAL '11' MONTH)`
 - support different field of ansi interval type in `coalesce` etc..

### How was this patch tested?
Added UT

Closes #33661 from AngersZh/SPARK-SPARK-36431.

Authored-by: Angerszh 
Signed-off-by: Max Gekk 
(cherry picked from commit 89d8a4eacfd09f67ad31bf1cbf7d4b88de3b1e24)
Signed-off-by: Max Gekk 
---
 .../sql/catalyst/analysis/AnsiTypeCoercion.scala   |  5 ++
 .../spark/sql/catalyst/analysis/TypeCoercion.scala |  5 ++
 .../sql/catalyst/analysis/TypeCoercionSuite.scala  | 47 +++
 .../test/resources/sql-tests/inputs/interval.sql   | 13 +++
 .../sql-tests/results/ansi/interval.sql.out| 98 +-
 .../resources/sql-tests/results/interval.sql.out   | 98 +-
 6 files changed, 258 insertions(+), 8 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala
index 457dc10..f03296f 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala
@@ -120,6 +120,11 @@ object AnsiTypeCoercion extends TypeCoercionBase {
 case (_: TimestampType, _: DateType) | (_: DateType, _: TimestampType) =>
   Some(TimestampType)
 
+case (t1: DayTimeIntervalType, t2: DayTimeIntervalType) =>
+  Some(DayTimeIntervalType(t1.startField.min(t2.startField), 
t1.endField.max(t2.endField)))
+case (t1: YearMonthIntervalType, t2: YearMonthIntervalType) =>
+  Some(YearMonthIntervalType(t1.startField.min(t2.startField), 
t1.endField.max(t2.endField)))
+
 case (t1, t2) => findTypeForComplex(t1, t2, findTightestCommonType)
   }
 
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 42c10e8..db6f499 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
@@ -867,6 +867,11 @@ object TypeCoercion extends TypeCoercionBase {
   case (_: TimestampType, _: DateType) | (_: DateType, _: TimestampType) =>
 Some(TimestampType)
 
+  case (t1: DayTimeIntervalType, t2: DayTimeIntervalType) =>
+Some(DayTimeIntervalType(t1.startField.min(t2.startField), 
t1.endField.max(t2.endField)))
+  case (t1: YearMonthIntervalType, t2: YearMonthIntervalType) =>
+Some(YearMonthIntervalType(t1.startField.min(t2.startField), 
t1.endField.max(t2.endField)))
+
   case (_: TimestampNTZType, _: DateType) | (_: DateType, _: 
TimestampNTZType) =>
 Some(TimestampNTZType)
 
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 602daf8..6a7d7ef 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
@@ -18,6 +18,7 @@
 package org.apache.spark.sql.catalyst.analysis
 
 import java.sql.Timestamp
+import java.time.{Duration, Period}
 
 import org.apache.spark.internal.config.Tests.IS_TESTING
 import org.apache.spark.sql.catalyst.analysis.TypeCoercion._
@@ -1604,6 +1605,52 @@ class TypeCoercionSuite extends AnalysisTest {
 ruleTest(TypeCoercion.IntegralDivision, IntegralDivide(2, 1L),
   IntegralDivide(Cast(2, LongTy

[spark] branch master updated (f7c85b8 -> bbf988b)

2021-08-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from f7c85b8  [SPARK-36456][CORE][SQL][SS] Clean up compilation warnings 
related to `method closeQuietly in class IOUtils is deprecated`
 add bbf988b  [SPARK-36468][SQL][DOCS] Update docs about ANSI interval 
literals

No new revisions were added by this update.

Summary of changes:
 docs/sql-ref-literals.md | 91 +---
 1 file changed, 70 insertions(+), 21 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36468][SQL][DOCS] Update docs about ANSI interval literals

2021-08-11 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 8dbcbeb  [SPARK-36468][SQL][DOCS] Update docs about ANSI interval 
literals
8dbcbeb is described below

commit 8dbcbebc3604f01b865215f2d02eb823cad582ec
Author: Max Gekk 
AuthorDate: Wed Aug 11 13:38:39 2021 +0300

[SPARK-36468][SQL][DOCS] Update docs about ANSI interval literals

### What changes were proposed in this pull request?
In the PR, I propose to update the doc page 
https://spark.apache.org/docs/latest/sql-ref-literals.html#interval-literal, 
and describe formats of ANSI interval literals.

https://user-images.githubusercontent.com/1580697/128988454-7a6ac435-409b-4961-9b79-ebecfb141d5e.png";>
https://user-images.githubusercontent.com/1580697/128912018-a4ea3ee5-f252-49c7-a90e-5beaf7ac868f.png";>

### Why are the changes needed?
To improve UX with Spark SQL, and inform users about recently added ANSI 
interval literals.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually checked the generated docs:
```
$ SKIP_API=1 SKIP_RDOC=1 SKIP_PYTHONDOC=1 SKIP_SCALADOC=1 bundle exec 
jekyll build
```

Closes #33693 from MaxGekk/doc-ansi-interval-literals.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit bbf988bd73b00d18dd1d443f225b3915a2c4433f)
Signed-off-by: Max Gekk 
---
 docs/sql-ref-literals.md | 91 +---
 1 file changed, 70 insertions(+), 21 deletions(-)

diff --git a/docs/sql-ref-literals.md b/docs/sql-ref-literals.md
index b23c63a..355ac69 100644
--- a/docs/sql-ref-literals.md
+++ b/docs/sql-ref-literals.md
@@ -439,13 +439,78 @@ SELECT TIMESTAMP '1997-01' AS col;
 
 An interval literal is used to specify a fixed period of time.
 
+ ANSI style
+
+The ANSI SQL standard defines interval literals in the form:
+```sql
+INTERVAL [  ]  
+```
+where `` can be a single field or in the field-to-field 
form:
+```sql
+ ::=  TO  | 
+```
+The field name is case-insensitive, and can be one of `YEAR`, `MONTH`, `DAY`, 
`HOUR`, `MINUTE` and `SECOND`.
+
+An interval literal can have either year-month or day-time interval type. The 
interval sub-type defines format of ``:
+```sql
+ ::=  [  ] {  |  } 
+ ::=  [   ] | 

+ ::=  | 
+ ::=  [   [  
 [   ] ] ]
+ ::=  [   [   ] ]
+  |  [   ]
+  | 
+```
+
+Supported year-month interval literals and theirs formats:
+
+|``|Interval string pattern|An instance of the literal|
+|-|---||
+|YEAR|`[+|-]'[+|-]y'`|`INTERVAL -'2021' YEAR`|
+|YEAR TO MONTH|`[+|-]'[+|-]y-m'`|`INTERVAL '-2021-07' YEAR TO MONTH`|
+|MONTH|`[+|-]'[+|-]m'`|`interval '10' month`|
+
+Formats of supported day-time interval literals:
+
+|``|Interval string pattern|An instance of the literal|
+|-||---|
+|DAY|`[+|-]'[+|-]d'`|`INTERVAL -'100' DAY`|
+|DAY TO HOUR|`[+|-]'[+|-]d h'`|`INTERVAL '-100 10' DAY TO HOUR`|
+|DAY TO MINUTE|`[+|-]'[+|-]d h:m'`|`INTERVAL '100 10:30' DAY TO MINUTE`|
+|DAY TO SECOND|`[+|-]'[+|-]d h:m:s.n'`|`INTERVAL '100 10:30:40.99' DAY TO 
SECOND`|
+|HOUR|`[+|-]'[+|-]h'`|`INTERVAL '123' HOUR`|
+|HOUR TO MINUTE|`[+|-]'[+|-]h:m'`|`INTERVAL -'-123:10' HOUR TO MINUTE`|
+|HOUR TO SECOND|`[+|-]'[+|-]h:m:s.n'`|`INTERVAL '123:10:59' HOUR TO SECOND`|
+|MINUTE|`[+|-]'[+|-]m'`|`interval '1000' minute`|
+|MINUTE TO SECOND|`[+|-]'[+|-]m:s.n'`|`INTERVAL '1000:01.001' MINUTE TO 
SECOND`|
+|SECOND|`[+|-]'[+|-]s.n'`|`INTERVAL '1000.01' SECOND`|
+
+# Examples
+
+```sql
+SELECT INTERVAL '2-3' YEAR TO MONTH AS col;
+++
+|col |
+++
+|INTERVAL '2-3' YEAR TO MONTH|
+++
+
+SELECT INTERVAL -'20 15:40:32.9989' DAY TO SECOND AS col;
+++
+|col |
+++
+|INTERVAL '-20 15:40:32.998999' DAY TO SECOND|
+++
+```
+
+ Multi-units style
+
 ```sql
 INTERVAL interval_value interval_unit [ interval_value interval_unit ... ] |
 INTERVAL 'interval_value interval_unit [ interval_value interval_unit ... ]' |
-INTERVAL interval_string_value interval_unit TO interval_unit
 ```
 
- Parameters
+# Parameters
 
 * **interval_value**
 
@@ -453,10 +518,6 @@ INTERVAL interval_string_value interval_

[spark] branch master updated (7d82336 -> eb6be7f)

2021-08-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 7d82336  [SPARK-36428][SQL] the seconds parameter of make_timestamp 
should accept integer type
 add eb6be7f  [SPARK-36499][SQL][TESTS] Test Interval multiply / divide null

No new revisions were added by this update.

Summary of changes:
 .../test/resources/sql-tests/inputs/interval.sql   | 11 
 .../sql-tests/results/ansi/interval.sql.out| 71 +-
 .../resources/sql-tests/results/interval.sql.out   | 71 +-
 3 files changed, 151 insertions(+), 2 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36499][SQL][TESTS] Test Interval multiply / divide null

2021-08-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new eaf92be  [SPARK-36499][SQL][TESTS] Test Interval multiply / divide null
eaf92be is described below

commit eaf92bea99d1a6f4661718c484d167c7a08a37f6
Author: Gengliang Wang 
AuthorDate: Fri Aug 13 11:05:57 2021 +0300

[SPARK-36499][SQL][TESTS] Test Interval multiply / divide null

### What changes were proposed in this pull request?

Test the following valid operations:
```
year-month interval * null
null * year-month interval
year-month interval / null
```
and invalid operations:
```
null / interval
int / interval
```

### Why are the changes needed?

Improve test coverage

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Pass CI

Closes #33729 from gengliangwang/addTest.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit eb6be7f1ee076aeaa312f7a3ff0c88db516b793b)
Signed-off-by: Max Gekk 
---
 .../test/resources/sql-tests/inputs/interval.sql   | 11 
 .../sql-tests/results/ansi/interval.sql.out| 71 +-
 .../resources/sql-tests/results/interval.sql.out   | 71 +-
 3 files changed, 151 insertions(+), 2 deletions(-)

diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql 
b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
index 80df45d..be13a25 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
@@ -16,6 +16,16 @@ select interval '2 seconds' / 0;
 select interval '2 seconds' / null;
 select interval '2 seconds' * null;
 select null * interval '2 seconds';
+select interval '2' year / 0;
+select interval '2' year / null;
+select interval '2' year * null;
+select null * interval '2' year;
+
+-- invalid: divide by interval
+select 2 / interval '2' year;
+select 2 / interval '2' hour;
+select null / interval '2' year;
+select null / interval '2' hour;
 
 -- interval with a positive/negative sign
 select -interval '-1 month 1 day -1 second';
@@ -170,6 +180,7 @@ select
   null - interval '2' year,
   null + interval '2' hour,
   null - interval '2' hour;
+
 -- invalid: malformed interval string
 select interval '2' year + '3-3';
 select interval '2' year - '4';
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out 
b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
index b048105..4e784b0 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 223
+-- Number of queries: 231
 
 
 -- !query
@@ -118,6 +118,75 @@ NULL
 
 
 -- !query
+select interval '2' year / 0
+-- !query schema
+struct<>
+-- !query output
+java.lang.ArithmeticException
+/ by zero
+
+
+-- !query
+select interval '2' year / null
+-- !query schema
+struct<(INTERVAL '2' YEAR / NULL):interval year to month>
+-- !query output
+NULL
+
+
+-- !query
+select interval '2' year * null
+-- !query schema
+struct<(INTERVAL '2' YEAR * NULL):interval year to month>
+-- !query output
+NULL
+
+
+-- !query
+select null * interval '2' year
+-- !query schema
+struct<(INTERVAL '2' YEAR * NULL):interval year to month>
+-- !query output
+NULL
+
+
+-- !query
+select 2 / interval '2' year
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(2 / INTERVAL '2' YEAR)' due to data type mismatch: differing 
types in '(2 / INTERVAL '2' YEAR)' (int and interval year).; line 1 pos 7
+
+
+-- !query
+select 2 / interval '2' hour
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(2 / INTERVAL '02' HOUR)' due to data type mismatch: differing 
types in '(2 / INTERVAL '02' HOUR)' (int and interval hour).; line 1 pos 7
+
+
+-- !query
+select null / interval '2' year
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+cannot resolve '(NULL / INTERVAL '2' YEAR)' due to data type mismatch: 
differing types in '(NULL / INTERVAL '2' YEAR)' (void and interval year

[spark] branch master updated (eb6be7f -> 7fd3454)

2021-08-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from eb6be7f  [SPARK-36499][SQL][TESTS] Test Interval multiply / divide null
 add 7fd3454  [SPARK-36490][SQL] Make from_csv/to_csv to handle 
timestamp_ntz type properly

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/csv/UnivocityGenerator.scala | 12 +++-
 .../apache/spark/sql/catalyst/csv/UnivocityParser.scala | 11 +++
 .../results/timestampNTZ/timestamp-ansi.sql.out |  5 ++---
 .../sql-tests/results/timestampNTZ/timestamp.sql.out|  5 ++---
 .../scala/org/apache/spark/sql/CsvFunctionsSuite.scala  | 17 -
 5 files changed, 42 insertions(+), 8 deletions(-)

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



[spark] branch master updated (7fd3454 -> 41436b2)

2021-08-13 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 7fd3454  [SPARK-36490][SQL] Make from_csv/to_csv to handle 
timestamp_ntz type properly
 add 41436b2  [SPARK-36507][DOCS] Remove/Replace missing links to AMP Camp 
materials from index.md

No new revisions were added by this update.

Summary of changes:
 docs/index.md | 7 +++
 1 file changed, 3 insertions(+), 4 deletions(-)

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



[spark] branch master updated (8b8d91c -> 9b9db5a)

2021-08-16 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 8b8d91c  [SPARK-36465][SS] Dynamic gap duration in session window
 add 9b9db5a  [SPARK-36491][SQL] Make from_json/to_json to handle 
timestamp_ntz type properly

No new revisions were added by this update.

Summary of changes:
 .../apache/spark/sql/catalyst/json/JacksonGenerator.scala  | 12 
 .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 12 
 .../sql-tests/results/timestampNTZ/timestamp-ansi.sql.out  |  5 ++---
 .../sql-tests/results/timestampNTZ/timestamp.sql.out   |  5 ++---
 .../scala/org/apache/spark/sql/JsonFunctionsSuite.scala| 14 +-
 5 files changed, 41 insertions(+), 7 deletions(-)

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



[spark] branch master updated: [SPARK-36521][SQL] Disallow comparison between Interval and String

2021-08-16 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 26d6b95  [SPARK-36521][SQL] Disallow comparison between Interval and 
String
26d6b95 is described below

commit 26d6b952dcf7d387930701396de9cef679df7432
Author: Gengliang Wang 
AuthorDate: Mon Aug 16 22:41:14 2021 +0300

[SPARK-36521][SQL] Disallow comparison between Interval and String

### What changes were proposed in this pull request?

Disallow comparison between Interval and String in the default type 
coercion rules.

### Why are the changes needed?

If a binary comparison contains interval type and string type, we can't 
decide which
interval type the string should be promoted as. There are many possible 
interval
types, such as year interval, month interval, day interval, hour interval, 
etc.

### Does this PR introduce _any_ user-facing change?

No, the new interval type is not released yet.

### How was this patch tested?

Existing UT

Closes #33750 from gengliangwang/disallowCom.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/analysis/TypeCoercion.scala | 16 +++-
 .../test/resources/sql-tests/inputs/interval.sql   |  6 ++
 .../sql-tests/results/ansi/interval.sql.out| 56 +-
 .../resources/sql-tests/results/interval.sql.out   | 86 ++
 4 files changed, 148 insertions(+), 16 deletions(-)

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 23654af..863bdc0 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
@@ -862,6 +862,18 @@ object TypeCoercion extends TypeCoercionBase {
 case _ => None
   }
 
+  // Return whether a string literal can be promoted as the give data type in 
a binary comparison.
+  private def canPromoteAsInBinaryComparison(dt: DataType) = dt match {
+// If a binary comparison contains interval type and string type, we can't 
decide which
+// interval type the string should be promoted as. There are many possible 
interval
+// types, such as year interval, month interval, day interval, hour 
interval, etc.
+case _: YearMonthIntervalType | _: DayTimeIntervalType => false
+// There is no need to add `Cast` for comparison between strings.
+case _: StringType => false
+case _: AtomicType => true
+case _ => false
+  }
+
   /**
* This function determines the target type of a comparison operator when 
one operand
* is a String and the other is not. It also handles when one op is a Date 
and the
@@ -891,8 +903,8 @@ object TypeCoercion extends TypeCoercionBase {
 case (n: DecimalType, s: StringType) => Some(DoubleType)
 case (s: StringType, n: DecimalType) => Some(DoubleType)
 
-case (l: StringType, r: AtomicType) if r != StringType => Some(r)
-case (l: AtomicType, r: StringType) if l != StringType => Some(l)
+case (l: StringType, r: AtomicType) if canPromoteAsInBinaryComparison(r) 
=> Some(r)
+case (l: AtomicType, r: StringType) if canPromoteAsInBinaryComparison(l) 
=> Some(l)
 case (l, r) => None
   }
 
diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql 
b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
index 618cf16..279c5441 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
@@ -341,9 +341,15 @@ SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS;
 SELECT INTERVAL '1' DAY < '1';
 SELECT INTERVAL '1' DAY = '1';
 SELECT INTERVAL '1' DAY > '1';
+SELECT '1' < INTERVAL '1' DAY;
+SELECT '1' = INTERVAL '1' DAY;
+SELECT '1' > INTERVAL '1' DAY;
 SELECT INTERVAL '1' YEAR < '1';
 SELECT INTERVAL '1' YEAR = '1';
 SELECT INTERVAL '1' YEAR > '1';
+SELECT '1' < INTERVAL '1' YEAR;
+SELECT '1' = INTERVAL '1' YEAR;
+SELECT '1' > INTERVAL '1' YEAR;
 
 SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH);
 SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE);
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out 
b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
index e0bf076..1aa0920 100644
--- a/sql/core/src/test/resources

[spark] branch branch-3.2 updated: [SPARK-36521][SQL] Disallow comparison between Interval and String

2021-08-16 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 41e5144  [SPARK-36521][SQL] Disallow comparison between Interval and 
String
41e5144 is described below

commit 41e5144b53d21d4c67e35250594ee418bdfba136
Author: Gengliang Wang 
AuthorDate: Mon Aug 16 22:41:14 2021 +0300

[SPARK-36521][SQL] Disallow comparison between Interval and String

### What changes were proposed in this pull request?

Disallow comparison between Interval and String in the default type 
coercion rules.

### Why are the changes needed?

If a binary comparison contains interval type and string type, we can't 
decide which
interval type the string should be promoted as. There are many possible 
interval
types, such as year interval, month interval, day interval, hour interval, 
etc.

### Does this PR introduce _any_ user-facing change?

No, the new interval type is not released yet.

### How was this patch tested?

Existing UT

Closes #33750 from gengliangwang/disallowCom.

Authored-by: Gengliang Wang 
Signed-off-by: Max Gekk 
(cherry picked from commit 26d6b952dcf7d387930701396de9cef679df7432)
Signed-off-by: Max Gekk 
---
 .../spark/sql/catalyst/analysis/TypeCoercion.scala | 16 +++-
 .../test/resources/sql-tests/inputs/interval.sql   |  6 ++
 .../sql-tests/results/ansi/interval.sql.out| 56 +-
 .../resources/sql-tests/results/interval.sql.out   | 86 ++
 4 files changed, 148 insertions(+), 16 deletions(-)

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 23654af..863bdc0 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
@@ -862,6 +862,18 @@ object TypeCoercion extends TypeCoercionBase {
 case _ => None
   }
 
+  // Return whether a string literal can be promoted as the give data type in 
a binary comparison.
+  private def canPromoteAsInBinaryComparison(dt: DataType) = dt match {
+// If a binary comparison contains interval type and string type, we can't 
decide which
+// interval type the string should be promoted as. There are many possible 
interval
+// types, such as year interval, month interval, day interval, hour 
interval, etc.
+case _: YearMonthIntervalType | _: DayTimeIntervalType => false
+// There is no need to add `Cast` for comparison between strings.
+case _: StringType => false
+case _: AtomicType => true
+case _ => false
+  }
+
   /**
* This function determines the target type of a comparison operator when 
one operand
* is a String and the other is not. It also handles when one op is a Date 
and the
@@ -891,8 +903,8 @@ object TypeCoercion extends TypeCoercionBase {
 case (n: DecimalType, s: StringType) => Some(DoubleType)
 case (s: StringType, n: DecimalType) => Some(DoubleType)
 
-case (l: StringType, r: AtomicType) if r != StringType => Some(r)
-case (l: AtomicType, r: StringType) if l != StringType => Some(l)
+case (l: StringType, r: AtomicType) if canPromoteAsInBinaryComparison(r) 
=> Some(r)
+case (l: AtomicType, r: StringType) if canPromoteAsInBinaryComparison(l) 
=> Some(l)
 case (l, r) => None
   }
 
diff --git a/sql/core/src/test/resources/sql-tests/inputs/interval.sql 
b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
index 618cf16..279c5441 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/interval.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/interval.sql
@@ -341,9 +341,15 @@ SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS;
 SELECT INTERVAL '1' DAY < '1';
 SELECT INTERVAL '1' DAY = '1';
 SELECT INTERVAL '1' DAY > '1';
+SELECT '1' < INTERVAL '1' DAY;
+SELECT '1' = INTERVAL '1' DAY;
+SELECT '1' > INTERVAL '1' DAY;
 SELECT INTERVAL '1' YEAR < '1';
 SELECT INTERVAL '1' YEAR = '1';
 SELECT INTERVAL '1' YEAR > '1';
+SELECT '1' < INTERVAL '1' YEAR;
+SELECT '1' = INTERVAL '1' YEAR;
+SELECT '1' > INTERVAL '1' YEAR;
 
 SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH);
 SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE);
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out 
b/sql/core/src/test/resource

[spark] branch master updated (ea13c5a -> 82a3150)

2021-08-17 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from ea13c5a  [SPARK-36052][K8S][FOLLOWUP] Update config version to 3.2.0
 add 82a3150  [SPARK-36524][SQL] Common class for ANSI interval types

No new revisions were added by this update.

Summary of changes:
 .../avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala | 2 +-
 .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala   | 8 
 .../org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala | 2 +-
 .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala| 2 +-
 .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala| 4 ++--
 .../spark/sql/catalyst/expressions/collectionOperations.scala | 2 +-
 .../spark/sql/catalyst/expressions/datetimeExpressions.scala  | 2 +-
 .../main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala | 4 ++--
 .../main/scala/org/apache/spark/sql/types/AbstractDataType.scala  | 5 +
 .../scala/org/apache/spark/sql/types/DayTimeIntervalType.scala| 2 +-
 .../scala/org/apache/spark/sql/types/YearMonthIntervalType.scala  | 2 +-
 .../spark/sql/execution/datasources/csv/CSVFileFormat.scala   | 2 +-
 .../spark/sql/execution/datasources/json/JsonFileFormat.scala | 2 +-
 .../spark/sql/execution/datasources/orc/OrcFileFormat.scala   | 2 +-
 .../sql/execution/datasources/parquet/ParquetFileFormat.scala | 2 +-
 .../apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala  | 4 ++--
 .../spark/sql/execution/datasources/v2/json/JsonTable.scala   | 2 +-
 .../apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala  | 2 +-
 .../spark/sql/execution/datasources/v2/parquet/ParquetTable.scala | 2 +-
 .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala| 2 +-
 .../spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala| 5 ++---
 .../main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala  | 2 +-
 22 files changed, 33 insertions(+), 29 deletions(-)

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



[spark] branch branch-3.2 updated: [SPARK-36524][SQL] Common class for ANSI interval types

2021-08-17 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
 new 07c6976  [SPARK-36524][SQL] Common class for ANSI interval types
07c6976 is described below

commit 07c6976f79e418be8aed9bed8e7b396231a27c25
Author: Max Gekk 
AuthorDate: Tue Aug 17 12:27:56 2021 +0300

[SPARK-36524][SQL] Common class for ANSI interval types

### What changes were proposed in this pull request?
Add new type `AnsiIntervalType` to `AbstractDataType.scala`, and extend it 
by `YearMonthIntervalType` and by `DayTimeIntervalType`

### Why are the changes needed?
To improve code maintenance. The change will allow to replace checking of 
both `YearMonthIntervalType` and `DayTimeIntervalType` by a check of 
`AnsiIntervalType`, for instance:
```scala
case _: YearMonthIntervalType | _: DayTimeIntervalType => false
```
by
```scala
case _: AnsiIntervalType => false
```

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
By existing test suites.

Closes #33753 from MaxGekk/ansi-interval-type-trait.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
(cherry picked from commit 82a31508afffd089048e28276c75b5deb1ada47f)
Signed-off-by: Max Gekk 
---
 .../avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala | 2 +-
 .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala   | 8 
 .../org/apache/spark/sql/catalyst/analysis/AnsiTypeCoercion.scala | 2 +-
 .../org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala| 2 +-
 .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala| 4 ++--
 .../spark/sql/catalyst/expressions/collectionOperations.scala | 2 +-
 .../spark/sql/catalyst/expressions/datetimeExpressions.scala  | 2 +-
 .../main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala | 4 ++--
 .../main/scala/org/apache/spark/sql/types/AbstractDataType.scala  | 5 +
 .../scala/org/apache/spark/sql/types/DayTimeIntervalType.scala| 2 +-
 .../scala/org/apache/spark/sql/types/YearMonthIntervalType.scala  | 2 +-
 .../spark/sql/execution/datasources/csv/CSVFileFormat.scala   | 2 +-
 .../spark/sql/execution/datasources/json/JsonFileFormat.scala | 2 +-
 .../spark/sql/execution/datasources/orc/OrcFileFormat.scala   | 2 +-
 .../sql/execution/datasources/parquet/ParquetFileFormat.scala | 2 +-
 .../apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala  | 4 ++--
 .../spark/sql/execution/datasources/v2/json/JsonTable.scala   | 2 +-
 .../apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala  | 2 +-
 .../spark/sql/execution/datasources/v2/parquet/ParquetTable.scala | 2 +-
 .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala| 2 +-
 .../spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala| 5 ++---
 .../main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala  | 2 +-
 22 files changed, 33 insertions(+), 29 deletions(-)

diff --git 
a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala 
b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
index 68b393e..5b8afe8 100644
--- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
+++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala
@@ -71,7 +71,7 @@ private[sql] object AvroUtils extends Logging {
   }
 
   def supportsDataType(dataType: DataType): Boolean = dataType match {
-case _: DayTimeIntervalType | _: YearMonthIntervalType => false
+case _: AnsiIntervalType => false
 
 case _: AtomicType => true
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 468986d..2f0a709 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -377,9 +377,9 @@ class Analyzer(override val catalogManager: CatalogManager)
 TimestampAddYMInterval(r, l)
   case (CalendarIntervalType, CalendarIntervalType) |
(_: DayTimeIntervalType, _: DayTimeIntervalType) => a
-  case (_: NullType, _: DayTimeIntervalType | _: 
YearMonthIntervalType) =>
+  case (_: NullType, _: AnsiIntervalType) =>
 a.copy(left = Cast(a.left, a.right.dataType))
-  case (_: DayTimeIntervalType | _: YearMonthIntervalType, _: 
NullType) =>
+  case (_: AnsiIntervalType, _: NullType) =>
 a.copy(right = Cast(a.right, a.left.dataType))
   case (DateType, CalendarIntervalType) => DateAddInterval(l, r, 
ansiEnabled = f)
   

[spark] branch master updated (013f2b7 -> 1235bd2)

2021-08-18 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git.


from 013f2b7  [SPARK-36512][UI][TESTS] Fix UISeleniumSuite in 
sql/hive-thriftserver
 add 1235bd2  [SPARK-36536][SQL] Use CAST for datetime in CSV/JSON by 
default

No new revisions were added by this update.

Summary of changes:
 .../spark/sql/catalyst/csv/CSVInferSchema.scala|  2 +-
 .../apache/spark/sql/catalyst/csv/CSVOptions.scala | 12 +--
 .../sql/catalyst/csv/UnivocityGenerator.scala  |  6 ++--
 .../spark/sql/catalyst/csv/UnivocityParser.scala   |  6 ++--
 .../spark/sql/catalyst/json/JSONOptions.scala  | 12 +--
 .../spark/sql/catalyst/json/JacksonGenerator.scala |  6 ++--
 .../spark/sql/catalyst/json/JacksonParser.scala|  6 ++--
 .../spark/sql/catalyst/json/JsonInferSchema.scala  |  2 +-
 .../spark/sql/catalyst/util/DateFormatter.scala|  8 +
 .../sql/catalyst/util/TimestampFormatter.scala | 24 ++
 .../sql/catalyst/csv/UnivocityParserSuite.scala|  4 +--
 .../sql/execution/datasources/csv/CSVSuite.scala   | 37 +-
 .../sql/execution/datasources/json/JsonSuite.scala | 37 --
 13 files changed, 139 insertions(+), 23 deletions(-)

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



[spark] branch master updated: [SPARK-36418][SPARK-36536][SQL][DOCS][FOLLOWUP] Update the SQL migration guide about using `CAST` in datetime parsing

2021-08-23 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 9f595c4  [SPARK-36418][SPARK-36536][SQL][DOCS][FOLLOWUP] Update the 
SQL migration guide about using `CAST` in datetime parsing
9f595c4 is described below

commit 9f595c4ce34728f5d8f943eadea8d85a548b2d41
Author: Max Gekk 
AuthorDate: Mon Aug 23 13:07:37 2021 +0300

[SPARK-36418][SPARK-36536][SQL][DOCS][FOLLOWUP] Update the SQL migration 
guide about using `CAST` in datetime parsing

### What changes were proposed in this pull request?
In the PR, I propose the update the SQL migration guide about the changes 
introduced by the PRs https://github.com/apache/spark/pull/33709 and 
https://github.com/apache/spark/pull/33769.

https://user-images.githubusercontent.com/1580697/130419710-640f20b3-6a38-4eb1-a6d6-2e069dc5665c.png";>

### Why are the changes needed?
To inform users about the upcoming changes in parsing datetime strings. 
This should help users to migrate on the new release.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
By generating the doc, and checking by eyes:
```
$ SKIP_API=1 SKIP_RDOC=1 SKIP_PYTHONDOC=1 SKIP_SCALADOC=1 bundle exec 
jekyll build
```

Closes #33809 from MaxGekk/datetime-cast-migr-guide.

Authored-by: Max Gekk 
Signed-off-by: Max Gekk 
---
 docs/sql-migration-guide.md | 20 
 1 file changed, 20 insertions(+)

diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md
index 7ad384f..47e7921 100644
--- a/docs/sql-migration-guide.md
+++ b/docs/sql-migration-guide.md
@@ -26,6 +26,26 @@ license: |
 
   - Since Spark 3.3, Spark turns a non-nullable schema into nullable for API 
`DataFrameReader.schema(schema: StructType).json(jsonDataset: Dataset[String])` 
and `DataFrameReader.schema(schema: StructType).csv(csvDataset: 
Dataset[String])` when the schema is specified by the user and contains 
non-nullable fields.
 
+  - Since Spark 3.3, when the date or timestamp pattern is not specified, 
Spark converts an input string to a date/timestamp using the `CAST` expression 
approach. The changes affect CSV/JSON datasources and parsing of partition 
values. In Spark 3.2 or earlier, when the date or timestamp pattern is not set, 
Spark uses the default patterns: `-MM-dd` for dates and `-MM-dd 
HH:mm:ss` for timestamps. After the changes, Spark still recognizes the pattern 
together with
+
+Date patterns:
+  * `[+-]*`
+  * `[+-]*-[m]m`
+  * `[+-]*-[m]m-[d]d`
+  * `[+-]*-[m]m-[d]d `
+  * `[+-]*-[m]m-[d]d *`
+  * `[+-]*-[m]m-[d]dT*`
+
+Timestamp patterns:
+  * `[+-]*`
+  * `[+-]*-[m]m`
+  * `[+-]*-[m]m-[d]d`
+  * `[+-]*-[m]m-[d]d `
+  * `[+-]*-[m]m-[d]d [h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+  * `[+-]*-[m]m-[d]dT[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+  * `[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+  * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us][zone_id]`
+
 ## Upgrading from Spark SQL 3.1 to 3.2
 
   - Since Spark 3.2, ADD FILE/JAR/ARCHIVE commands require each path to be 
enclosed by `"` or `'` if the path contains whitespaces.

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



[spark] branch master updated: [SPARK-36626][PYTHON][FOLLOW-UP] Use datetime.tzinfo instead of datetime.tzname()

2021-09-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new c6f3a13  [SPARK-36626][PYTHON][FOLLOW-UP] Use datetime.tzinfo instead 
of datetime.tzname()
c6f3a13 is described below

commit c6f3a13087a954d56ef671ecb82c8031a2f45d52
Author: Hyukjin Kwon 
AuthorDate: Mon Sep 6 17:16:52 2021 +0200

[SPARK-36626][PYTHON][FOLLOW-UP] Use datetime.tzinfo instead of 
datetime.tzname()

### What changes were proposed in this pull request?

This PR is a small followup of https://github.com/apache/spark/pull/33876 
which proposes to use `datetime.tzinfo` instead of `datetime.tzname` to see if 
timezome information is provided or not.

This way is consistent with other places such as:


https://github.com/apache/spark/blob/9c5bcac61ee56fbb271e890cc33f9a983612c5b0/python/pyspark/sql/types.py#L182


https://github.com/apache/spark/blob/9c5bcac61ee56fbb271e890cc33f9a983612c5b0/python/pyspark/sql/types.py#L1662

### Why are the changes needed?

In some cases, `datetime.tzname` can raise an exception 
(https://docs.python.org/3/library/datetime.html#datetime.datetime.tzname):

> ... raises an exception if the latter doesn’t return None or a string 
object,

I was able to reproduce this in Jenkins with setting 
`spark.sql.timestampType` to `TIMESTAMP_NTZ` by default:

```
==
ERROR: test_time_with_timezone (pyspark.sql.tests.test_serde.SerdeTests)
--
Traceback (most recent call last):
  File 
"/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/sql/tests/test_serde.py",
 line 92, in test_time_with_timezone
...
  File "/usr/lib/pypy3/lib-python/3/datetime.py", line 979, in tzname
raise NotImplementedError("tzinfo subclass must override tzname()")
NotImplementedError: tzinfo subclass must override tzname()
```

### Does this PR introduce _any_ user-facing change?

No to end users because it has not be released.
This is rather a safeguard to prevent potential breakage.

### How was this patch tested?

Manually tested.

Closes #33918 from HyukjinKwon/SPARK-36626-followup.

Authored-by: Hyukjin Kwon 
Signed-off-by: Max Gekk 
---
 python/pyspark/sql/types.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py
index 6cb8aec..e8b7411 100644
--- a/python/pyspark/sql/types.py
+++ b/python/pyspark/sql/types.py
@@ -1045,7 +1045,7 @@ def _infer_type(obj, infer_dict_as_struct=False, 
prefer_timestamp_ntz=False):
 if dataType is DecimalType:
 # the precision and scale of `obj` may be different from row to row.
 return DecimalType(38, 18)
-if dataType is TimestampType and prefer_timestamp_ntz and obj.tzname() is 
None:
+if dataType is TimestampType and prefer_timestamp_ntz and obj.tzinfo is 
None:
 return TimestampNTZType()
 elif dataType is not None:
 return dataType()

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



[spark] branch master updated: [SPARK-36675][SQL] Support ScriptTransformation for timestamp_ntz

2021-09-06 Thread maxgekk
This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
 new 0ab0cb1  [SPARK-36675][SQL] Support ScriptTransformation for 
timestamp_ntz
0ab0cb1 is described below

commit 0ab0cb108d64c95c0d46075c9c30d735d74a3b0d
Author: Kousuke Saruta 
AuthorDate: Mon Sep 6 20:58:07 2021 +0200

[SPARK-36675][SQL] Support ScriptTransformation for timestamp_ntz

### What changes were proposed in this pull request?

This PR aims to support `ScriptTransformation` for `timestamp_ntz`.
In the current master, it doesn't work.
```
spark.sql("SELECT transform(col1) USING 'cat' AS (col1 timestamp_ntz) FROM 
VALUES timestamp_ntz'2021-09-06 20:19:13' t").show(false)
21/09/06 22:03:55 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
org.apache.spark.SparkException: SparkScriptTransformation without serde 
does not support TimestampNTZType$ as output data type
at 
org.apache.spark.sql.errors.QueryExecutionErrors$.outputDataTypeUnsupportedByNodeWithoutSerdeError(QueryExecutionErrors.scala:1740)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec.$anonfun$outputFieldWriters$1(BaseScriptTransformationExec.scala:245)
at scala.collection.immutable.List.map(List.scala:293)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec.org$apache$spark$sql$execution$BaseScriptTransformationExec$$outputFieldWriters(BaseScriptTransformationExec.scala:194)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec.org$apache$spark$sql$execution$BaseScriptTransformationExec$$outputFieldWriters$(BaseScriptTransformationExec.scala:194)
at 
org.apache.spark.sql.execution.SparkScriptTransformationExec.org$apache$spark$sql$execution$BaseScriptTransformationExec$$outputFieldWriters$lzycompute(SparkScriptTransformationExec.scala:38)
at 
org.apache.spark.sql.execution.SparkScriptTransformationExec.org$apache$spark$sql$execution$BaseScriptTransformationExec$$outputFieldWriters(SparkScriptTransformationExec.scala:38)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec$$anon$1.$anonfun$processRowWithoutSerde$1(BaseScriptTransformationExec.scala:121)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec$$anon$1.next(BaseScriptTransformationExec.scala:162)
at 
org.apache.spark.sql.execution.BaseScriptTransformationExec$$anon$1.next(BaseScriptTransformationExec.scala:113)
```

### Why are the changes needed?

For better usability.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New test.

Closes #33920 from sarutak/script-transformation-timestamp-ntz.

Authored-by: Kousuke Saruta 
Signed-off-by: Max Gekk 
---
 .../spark/sql/execution/BaseScriptTransformationExec.scala |  3 +++
 .../sql/execution/BaseScriptTransformationSuite.scala  | 14 ++
 2 files changed, 17 insertions(+)

diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
index fc3a124..6040050 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala
@@ -220,6 +220,9 @@ trait BaseScriptTransformationExec extends UnaryExecNode {
 UTF8String.fromString(data),
 DateTimeUtils.getZoneId(conf.sessionLocalTimeZone))
 .map(DateTimeUtils.toJavaTimestamp).orNull, converter)
+  case TimestampNTZType =>
+wrapperConvertException(data => 
DateTimeUtils.stringToTimestampWithoutTimeZone(
+  
UTF8String.fromString(data)).map(DateTimeUtils.microsToLocalDateTime).orNull, 
converter)
   case CalendarIntervalType => wrapperConvertException(
 data => IntervalUtils.stringToInterval(UTF8String.fromString(data)),
 converter)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
index 9d8fcda..488a0fd 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala
@@ -654,6 +654,20 @@ abstract class BaseScriptTransformationSuite extends 
SparkPlanTest with SQLTestU
 df.select($"ym", $"dt").collect())
 }
   }
+
+  test("SPARK-36675: TRANSFORM should support timestamp_ntz (no serde)") {
+val df = spark.sql("SELECT timestamp_nt

  1   2   3   4   5   6   7   8   9   10   >