From 92e459baaccec2d3d211ae8f6a1042df80b4f270 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 17:32:31 +0800 Subject: [PATCH 01/16] Revert "[SPARK-29951][SQL] Make the behavior of Postgre dialect independent of ansi mode config" This reverts commit 23b3c4fafdf37a482b3f823a5701d99d96236519. --- docs/sql-keywords.md | 7 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 ++- .../sql/catalyst/parser/AstBuilder.scala | 16 +- .../sql/catalyst/parser/ParseDriver.scala | 12 +- .../apache/spark/sql/internal/SQLConf.scala | 4 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 + .../sql-tests/results/postgreSQL/text.sql.out | 160 ++++++++++-------- .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + 8 files changed, 111 insertions(+), 119 deletions(-) mode change 100755 => 100644 sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 9ad6984913a27..7c1f5028f4610 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,13 +19,12 @@ license: | limitations under the License. --- -When `spark.sql.dialect=PostgreSQL` or keep default `spark.sql.dialect=Spark` with setting `spark.sql.dialect.spark.ansi.enabled` to true, Spark SQL will use the ANSI mode parser. -In this mode, Spark SQL has two kinds of keywords: +When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When the ANSI mode is disabled, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when the ANSI mode enabled. +When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. By default `spark.sql.dialect.spark.ansi.enabled` is false. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index f894836d64233..418855ffb719f 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -58,9 +58,9 @@ grammar SqlBase; } /** - * When true, the behavior of keywords follows ANSI SQL standard. + * When true, ANSI SQL parsing mode is enabled. */ - public boolean SQL_standard_keyword_behavior = false; + public boolean ansi = false; } singleStatement @@ -766,7 +766,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' + | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | identifier '->' expression #lambda | '(' identifier (',' identifier)+ ')' '->' expression #lambda @@ -810,7 +810,7 @@ booleanValue interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? - | {SQL_standard_keyword_behavior}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) + | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval @@ -930,12 +930,6 @@ qualifiedNameList : qualifiedName (',' qualifiedName)* ; -functionName - : qualifiedName - | LEFT - | RIGHT - ; - qualifiedName : identifier ('.' identifier)* ; @@ -955,14 +949,14 @@ errorCapturingIdentifierExtra identifier : strictIdentifier - | {!SQL_standard_keyword_behavior}? strictNonReserved + | {!ansi}? strictNonReserved ; strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative - | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier - | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier + | {ansi}? ansiNonReserved #unquotedIdentifier + | {!ansi}? nonReserved #unquotedIdentifier ; quotedIdentifier @@ -981,7 +975,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1181,9 +1175,9 @@ ansiNonReserved | YEARS ; -// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `SQL_standard_keyword_behavior=true`. +// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. 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 858870a161417..3b5b44d926d0f 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 @@ -1589,7 +1589,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { // Create the function call. - val name = ctx.functionName.getText + val name = ctx.qualifiedName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) @@ -1599,8 +1599,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case expressions => expressions } - val function = UnresolvedFunction( - getFunctionIdentifier(ctx.functionName), arguments, isDistinct) + val function = UnresolvedFunction(visitFunctionName(ctx.qualifiedName), arguments, isDistinct) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { @@ -1640,17 +1639,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } - /** - * Get a function identifier consist by database (optional) and name. - */ - protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { - if (ctx.qualifiedName != null) { - visitFunctionName(ctx.qualifiedName) - } else { - FunctionIdentifier(ctx.getText, None) - } - } - /** * Create an [[LambdaFunction]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index b968848224c54..c19b7b978a01f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** @@ -89,20 +88,13 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = { logDebug(s"Parsing command: $command") - // When we use PostgreSQL dialect or use Spark dialect with setting - // `spark.sql.dialect.spark.ansi.enabled=true`, the parser will use ANSI SQL standard keywords. - val SQLStandardKeywordBehavior = conf.dialect match { - case Dialect.POSTGRESQL => true - case Dialect.SPARK => conf.dialectSparkAnsiEnabled - } - val lexer = new SqlBaseLexer(new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled lexer.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - lexer.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior + lexer.SQL_standard_keyword_behavior = conf.dialectSparkAnsiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -112,7 +104,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled parser.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - parser.SQL_standard_keyword_behavior = SQLStandardKeywordBehavior + parser.SQL_standard_keyword_behavior = conf.dialectSparkAnsiEnabled try { try { 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 f535792489fb4..743d88d7e457c 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 @@ -2555,9 +2555,7 @@ class SQLConf extends Serializable with Logging { def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def dialect: Dialect.Value = Dialect.withName(getConf(DIALECT)) - - def usePostgreSQLDialect: Boolean = dialect == Dialect.POSTGRESQL + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index 05953123da86f..a1fe95462ecae 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -44,7 +44,11 @@ select concat_ws(',',10,20,null,30); select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); +-- [SPARK-28036] Built-in udf left/right has inconsistent behavior +-- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode +set spark.sql.dialect.spark.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; +set spark.sql.dialect.spark.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out old mode 100755 new mode 100644 index 311b0eb5a5844..cac1e7ee5ab12 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 42 +-- Number of queries: 44 -- !query 0 @@ -151,10 +151,18 @@ edcba -- !query 18 -select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i +set spark.sql.dialect.spark.ansi.enabled=false -- !query 18 schema -struct +struct -- !query 18 output +spark.sql.dialect.spark.ansi.enabled false + + +-- !query 19 +select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i +-- !query 19 schema +struct +-- !query 19 output -5 -4 -3 @@ -168,192 +176,200 @@ struct 5 ahoj ahoj --- !query 19 +-- !query 20 +set spark.sql.dialect.spark.ansi.enabled=true +-- !query 20 schema +struct +-- !query 20 output +spark.sql.dialect.spark.ansi.enabled true + + +-- !query 21 /* * format */ select format_string(NULL) --- !query 19 schema +-- !query 21 schema struct --- !query 19 output +-- !query 21 output NULL --- !query 20 +-- !query 22 select format_string('Hello') --- !query 20 schema +-- !query 22 schema struct --- !query 20 output +-- !query 22 output Hello --- !query 21 +-- !query 23 select format_string('Hello %s', 'World') --- !query 21 schema +-- !query 23 schema struct --- !query 21 output +-- !query 23 output Hello World --- !query 22 +-- !query 24 select format_string('Hello %%') --- !query 22 schema +-- !query 24 schema struct --- !query 22 output +-- !query 24 output Hello % --- !query 23 +-- !query 25 select format_string('Hello %%%%') --- !query 23 schema +-- !query 25 schema struct --- !query 23 output +-- !query 25 output Hello %% --- !query 24 +-- !query 26 select format_string('Hello %s %s', 'World') --- !query 24 schema +-- !query 26 schema struct<> --- !query 24 output +-- !query 26 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 25 +-- !query 27 select format_string('Hello %s') --- !query 25 schema +-- !query 27 schema struct<> --- !query 25 output +-- !query 27 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 26 +-- !query 28 select format_string('Hello %x', 20) --- !query 26 schema +-- !query 28 schema struct --- !query 26 output +-- !query 28 output Hello 14 --- !query 27 +-- !query 29 select format_string('%1$s %3$s', 1, 2, 3) --- !query 27 schema +-- !query 29 schema struct --- !query 27 output +-- !query 29 output 1 3 --- !query 28 +-- !query 30 select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 28 schema +-- !query 30 schema struct --- !query 28 output +-- !query 30 output 1 12 --- !query 29 +-- !query 31 select format_string('%1$s %4$s', 1, 2, 3) --- !query 29 schema +-- !query 31 schema struct<> --- !query 29 output +-- !query 31 output java.util.MissingFormatArgumentException Format specifier '%4$s' --- !query 30 +-- !query 32 select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 30 schema +-- !query 32 schema struct<> --- !query 30 output +-- !query 32 output java.util.MissingFormatArgumentException Format specifier '%13$s' --- !query 31 +-- !query 33 select format_string('%0$s', 'Hello') --- !query 31 schema +-- !query 33 schema struct --- !query 31 output +-- !query 33 output Hello --- !query 32 +-- !query 34 select format_string('Hello %s %1$s %s', 'World', 'Hello again') --- !query 32 schema +-- !query 34 schema struct --- !query 32 output +-- !query 34 output Hello World World Hello again --- !query 33 +-- !query 35 select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') --- !query 33 schema +-- !query 35 schema struct --- !query 33 output +-- !query 35 output Hello World Hello again, Hello again Hello again --- !query 34 +-- !query 36 select format_string('>>%10s<<', 'Hello') --- !query 34 schema +-- !query 36 schema struct>%10s<<, Hello):string> --- !query 34 output +-- !query 36 output >> Hello<< --- !query 35 +-- !query 37 select format_string('>>%10s<<', NULL) --- !query 35 schema +-- !query 37 schema struct>%10s<<, NULL):string> --- !query 35 output +-- !query 37 output >> null<< --- !query 36 +-- !query 38 select format_string('>>%10s<<', '') --- !query 36 schema +-- !query 38 schema struct>%10s<<, ):string> --- !query 36 output +-- !query 38 output >> << --- !query 37 +-- !query 39 select format_string('>>%-10s<<', '') --- !query 37 schema +-- !query 39 schema struct>%-10s<<, ):string> --- !query 37 output +-- !query 39 output >> << --- !query 38 +-- !query 40 select format_string('>>%-10s<<', 'Hello') --- !query 38 schema +-- !query 40 schema struct>%-10s<<, Hello):string> --- !query 38 output +-- !query 40 output >>Hello << --- !query 39 +-- !query 41 select format_string('>>%-10s<<', NULL) --- !query 39 schema +-- !query 41 schema struct>%-10s<<, NULL):string> --- !query 39 output +-- !query 41 output >>null << --- !query 40 +-- !query 42 select format_string('>>%1$10s<<', 'Hello') --- !query 40 schema +-- !query 42 schema struct>%1$10s<<, Hello):string> --- !query 40 output +-- !query 42 output >> Hello<< --- !query 41 +-- !query 43 DROP TABLE TEXT_TBL --- !query 41 schema +-- !query 43 schema struct<> --- !query 41 output +-- !query 43 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index e6dcf0b86308a..cd4b43f450c0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -332,6 +332,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) + localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) From 8c32f6d75dc9da87aad0da712b67eaf88aa9253c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 17:50:37 +0800 Subject: [PATCH 02/16] Revert "[SPARK-29837][SQL] PostgreSQL dialect: cast to boolean" This reverts commit fe1f456b200075bef476b5d8eca2be505b486033. --- .../catalyst/analysis/PostgreSQLDialect.scala | 12 +-- .../spark/sql/catalyst/expressions/Cast.scala | 10 +-- .../PostgreCastStringToBoolean.scala | 80 +++++++++++++++++++ .../expressions/postgreSQL/CastSuite.scala | 74 +++++++---------- .../results/postgreSQL/boolean.sql.out | 60 ++++++-------- .../sql/PostgreSQLDialectQuerySuite.scala | 2 +- 6 files changed, 146 insertions(+), 92 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala index e7f0e571804d3..934e53703e241 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Cast -import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastToBoolean +import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastStringToBoolean import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -27,19 +27,19 @@ import org.apache.spark.sql.types.{BooleanType, StringType} object PostgreSQLDialect { val postgreSQLDialectRules: List[Rule[LogicalPlan]] = - CastToBoolean :: + CastStringToBoolean :: Nil - object CastToBoolean extends Rule[LogicalPlan] with Logging { + object CastStringToBoolean extends Rule[LogicalPlan] with Logging { override def apply(plan: LogicalPlan): LogicalPlan = { // The SQL configuration `spark.sql.dialect` can be changed in runtime. // To make sure the configuration is effective, we have to check it during rule execution. val conf = SQLConf.get if (conf.usePostgreSQLDialect) { plan.transformExpressions { - case Cast(child, dataType, timeZoneId) - if child.dataType != BooleanType && dataType == BooleanType => - PostgreCastToBoolean(child, timeZoneId) + case Cast(child, dataType, _) + if dataType == BooleanType && child.dataType == StringType => + PostgreCastStringToBoolean(child) } } else { plan diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index a871a746d64ff..807dda274edba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -276,7 +276,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def needsTimeZone: Boolean = Cast.needsTimeZone(child.dataType, dataType) // [[func]] assumes the input is no longer null because eval already does the null check. - @inline protected def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) + @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) private lazy val dateFormatter = DateFormatter(zoneId) private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) @@ -387,7 +387,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } // UDFToBoolean - protected[this] def castToBoolean(from: DataType): Any => Any = from match { + private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => { if (StringUtils.isTrueString(s)) { @@ -794,7 +794,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit } } - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = child.genCode(ctx) val nullSafeCast = nullSafeCastFunction(child.dataType, dataType, ctx) @@ -804,7 +804,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit // The function arguments are: `input`, `result` and `resultIsNull`. We don't need `inputIsNull` // in parameter list, because the returned code will be put in null safe evaluation region. - protected type CastFunction = (ExprValue, ExprValue, ExprValue) => Block + private[this] type CastFunction = (ExprValue, ExprValue, ExprValue) => Block private[this] def nullSafeCastFunction( from: DataType, @@ -1254,7 +1254,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit private[this] def timestampToDoubleCode(ts: ExprValue): Block = code"$ts / (double)$MICROS_PER_SECOND" - protected[this] def castToBooleanCode(from: DataType): CastFunction = from match { + private[this] def castToBooleanCode(from: DataType): CastFunction = from match { case StringType => val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" (c, evPrim, evNull) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala new file mode 100644 index 0000000000000..0e87707d01e47 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.postgreSQL + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, JavaCode} +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils +import org.apache.spark.sql.types.{BooleanType, DataType, StringType} +import org.apache.spark.unsafe.types.UTF8String + +case class PostgreCastStringToBoolean(child: Expression) + extends UnaryExpression with NullIntolerant { + + override def checkInputDataTypes(): TypeCheckResult = { + if (child.dataType == StringType) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure( + s"The expression ${getClass.getSimpleName} only accepts string input data type") + } + } + + override def nullSafeEval(input: Any): Any = { + val s = input.asInstanceOf[UTF8String].trim().toLowerCase() + if (StringUtils.isTrueString(s)) { + true + } else if (StringUtils.isFalseString(s)) { + false + } else { + null + } + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" + val eval = child.genCode(ctx) + val javaType = JavaCode.javaType(dataType) + val preprocessedString = ctx.freshName("preprocessedString") + val castCode = + code""" + boolean ${ev.isNull} = ${eval.isNull}; + $javaType ${ev.value} = false; + if (!${eval.isNull}) { + UTF8String $preprocessedString = ${eval.value}.trim().toLowerCase(); + if ($stringUtils.isTrueString($preprocessedString)) { + ${ev.value} = true; + } else if ($stringUtils.isFalseString($preprocessedString)) { + ${ev.value} = false; + } else { + ${ev.isNull} = true; + } + } + """ + ev.copy(code = eval.code + castCode) + } + + override def dataType: DataType = BooleanType + + override def nullable: Boolean = true + + override def toString: String = s"PostgreCastStringToBoolean($child as ${dataType.simpleString})" + + override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala index 6c5218b379f31..175904da21969 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala @@ -16,58 +16,44 @@ */ package org.apache.spark.sql.catalyst.expressions.postgreSQL -import java.sql.{Date, Timestamp} - import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} class CastSuite extends SparkFunSuite with ExpressionEvalHelper { - private def checkPostgreCastToBoolean(v: Any, expected: Any): Unit = { - checkEvaluation(PostgreCastToBoolean(Literal(v), None), expected) + private def checkPostgreCastStringToBoolean(v: Any, expected: Any): Unit = { + checkEvaluation(PostgreCastStringToBoolean(Literal(v)), expected) } test("cast string to boolean") { - checkPostgreCastToBoolean("true", true) - checkPostgreCastToBoolean("tru", true) - checkPostgreCastToBoolean("tr", true) - checkPostgreCastToBoolean("t", true) - checkPostgreCastToBoolean("tRUe", true) - checkPostgreCastToBoolean(" tRue ", true) - checkPostgreCastToBoolean(" tRu ", true) - checkPostgreCastToBoolean("yes", true) - checkPostgreCastToBoolean("ye", true) - checkPostgreCastToBoolean("y", true) - checkPostgreCastToBoolean("1", true) - checkPostgreCastToBoolean("on", true) - - checkPostgreCastToBoolean("false", false) - checkPostgreCastToBoolean("fals", false) - checkPostgreCastToBoolean("fal", false) - checkPostgreCastToBoolean("fa", false) - checkPostgreCastToBoolean("f", false) - checkPostgreCastToBoolean(" fAlse ", false) - checkPostgreCastToBoolean(" fAls ", false) - checkPostgreCastToBoolean(" FAlsE ", false) - checkPostgreCastToBoolean("no", false) - checkPostgreCastToBoolean("n", false) - checkPostgreCastToBoolean("0", false) - checkPostgreCastToBoolean("off", false) - checkPostgreCastToBoolean("of", false) + checkPostgreCastStringToBoolean("true", true) + checkPostgreCastStringToBoolean("tru", true) + checkPostgreCastStringToBoolean("tr", true) + checkPostgreCastStringToBoolean("t", true) + checkPostgreCastStringToBoolean("tRUe", true) + checkPostgreCastStringToBoolean(" tRue ", true) + checkPostgreCastStringToBoolean(" tRu ", true) + checkPostgreCastStringToBoolean("yes", true) + checkPostgreCastStringToBoolean("ye", true) + checkPostgreCastStringToBoolean("y", true) + checkPostgreCastStringToBoolean("1", true) + checkPostgreCastStringToBoolean("on", true) - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("o"), None).eval()) - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal("abc"), None).eval()) - intercept[IllegalArgumentException](PostgreCastToBoolean(Literal(""), None).eval()) - } + checkPostgreCastStringToBoolean("false", false) + checkPostgreCastStringToBoolean("fals", false) + checkPostgreCastStringToBoolean("fal", false) + checkPostgreCastStringToBoolean("fa", false) + checkPostgreCastStringToBoolean("f", false) + checkPostgreCastStringToBoolean(" fAlse ", false) + checkPostgreCastStringToBoolean(" fAls ", false) + checkPostgreCastStringToBoolean(" FAlsE ", false) + checkPostgreCastStringToBoolean("no", false) + checkPostgreCastStringToBoolean("n", false) + checkPostgreCastStringToBoolean("0", false) + checkPostgreCastStringToBoolean("off", false) + checkPostgreCastStringToBoolean("of", false) - test("unsupported data types to cast to boolean") { - assert(PostgreCastToBoolean(Literal(new Timestamp(1)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(new Date(1)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toLong), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toShort), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toByte), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(BigDecimal(1.0)), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toDouble), None).checkInputDataTypes().isFailure) - assert(PostgreCastToBoolean(Literal(1.toFloat), None).checkInputDataTypes().isFailure) + checkPostgreCastStringToBoolean("o", null) + checkPostgreCastStringToBoolean("abc", null) + checkPostgreCastStringToBoolean("", null) } } diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index e5f3425efc458..203806d43368a 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -53,10 +53,9 @@ true -- !query 6 SELECT boolean('test') AS error -- !query 6 schema -struct<> +struct -- !query 6 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: test +NULL -- !query 7 @@ -70,10 +69,9 @@ false -- !query 8 SELECT boolean('foo') AS error -- !query 8 schema -struct<> +struct -- !query 8 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: foo +NULL -- !query 9 @@ -95,10 +93,9 @@ true -- !query 11 SELECT boolean('yeah') AS error -- !query 11 schema -struct<> +struct -- !query 11 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: yeah +NULL -- !query 12 @@ -120,10 +117,9 @@ false -- !query 14 SELECT boolean('nay') AS error -- !query 14 schema -struct<> +struct -- !query 14 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: nay +NULL -- !query 15 @@ -153,28 +149,25 @@ false -- !query 18 SELECT boolean('o') AS error -- !query 18 schema -struct<> +struct -- !query 18 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: o +NULL -- !query 19 SELECT boolean('on_') AS error -- !query 19 schema -struct<> +struct -- !query 19 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: on_ +NULL -- !query 20 SELECT boolean('off_') AS error -- !query 20 schema -struct<> +struct -- !query 20 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: off_ +NULL -- !query 21 @@ -188,10 +181,9 @@ true -- !query 22 SELECT boolean('11') AS error -- !query 22 schema -struct<> +struct -- !query 22 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: 11 +NULL -- !query 23 @@ -205,19 +197,17 @@ false -- !query 24 SELECT boolean('000') AS error -- !query 24 schema -struct<> +struct -- !query 24 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: 000 +NULL -- !query 25 SELECT boolean('') AS error -- !query 25 schema -struct<> +struct -- !query 25 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: +NULL -- !query 26 @@ -320,19 +310,17 @@ true false -- !query 38 SELECT boolean(string(' tru e ')) AS invalid -- !query 38 schema -struct<> +struct -- !query 38 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: tru e +NULL -- !query 39 SELECT boolean(string('')) AS invalid -- !query 39 schema -struct<> +struct -- !query 39 output -java.lang.IllegalArgumentException -invalid input syntax for type boolean: +NULL -- !query 40 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala index 7056f483609a9..1354dcfda45fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala @@ -36,7 +36,7 @@ class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession { } Seq("o", "abc", "").foreach { input => - intercept[IllegalArgumentException](sql(s"select cast('$input' as boolean)").collect()) + checkAnswer(sql(s"select cast('$input' as boolean)"), Row(null)) } } } From 848745f846aa50383d23f87c582759f2c7e3dac0 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 19:16:23 +0800 Subject: [PATCH 03/16] Revert "[SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"" This reverts commit 40ea4a11d7f1534023669f0b81faf5d398174e46. --- docs/sql-keywords.md | 8 +++--- .../spark/sql/catalyst/parser/SqlBase.g4 | 6 ++-- .../spark/sql/catalyst/expressions/Cast.scala | 4 +-- .../sql/catalyst/expressions/arithmetic.scala | 2 +- .../sql/catalyst/parser/ParseDriver.scala | 4 +-- .../apache/spark/sql/internal/SQLConf.scala | 28 +++++++++---------- .../encoders/ExpressionEncoderSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 4 +-- .../ArithmeticExpressionSuite.scala | 24 ++++++++-------- .../sql/catalyst/expressions/CastSuite.scala | 17 ++++------- .../expressions/DecimalExpressionSuite.scala | 4 +-- .../catalyst/expressions/ScalaUDFSuite.scala | 4 +-- .../parser/ExpressionParserSuite.scala | 10 +++---- .../parser/TableIdentifierParserSuite.scala | 2 +- .../sql-tests/inputs/postgreSQL/text.sql | 4 +-- .../sql-tests/results/postgreSQL/text.sql.out | 8 +++--- .../org/apache/spark/sql/DataFrameSuite.scala | 2 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 4 +-- .../ThriftServerQueryTestSuite.scala | 6 ++-- 19 files changed, 69 insertions(+), 74 deletions(-) diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 7c1f5028f4610..85dffd6ea6d66 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,15 +19,15 @@ license: | limitations under the License. --- -When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.dialect.spark.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. -By default `spark.sql.dialect.spark.ansi.enabled` is false. +By default `spark.sql.ansi.enabled` is false. Below is a list of all the keywords in Spark SQL. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 418855ffb719f..15be6da14f5ec 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -975,7 +975,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.dialect.spark.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1175,9 +1175,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.dialect.spark.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `spark.sql.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.dialect.spark.ansi.enabled=true`. +// Same definition as the one when `spark.sql.ansi.enabled=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 807dda274edba..fa27a48419dbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -602,7 +602,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit * Change the precision / scale in a given decimal to those set in `decimalType` (if any), * modifying `value` in-place and returning it if successful. If an overflow occurs, it * either returns null or throws an exception according to the value set for - * `spark.sql.dialect.spark.ansi.enabled`. + * `spark.sql.ansi.enabled`. * * NOTE: this modifies `value` in-place, so don't call it on external data. */ @@ -621,7 +621,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit /** * Create new `Decimal` with precision and scale given in `decimalType` (if any). - * If overflow occurs, if `spark.sql.dialect.spark.ansi.enabled` is false, null is returned; + * If overflow occurs, if `spark.sql.ansi.enabled` is false, null is returned; * otherwise, an `ArithmeticException` is thrown. */ private[this] def toPrecision(value: Decimal, decimalType: DecimalType): Decimal = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 7650fb07a61cd..82a8e6d80a0bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -150,7 +150,7 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant { sys.error("BinaryArithmetics must override either calendarIntervalMethod or genCode") // Name of the function for the exact version of this expression in [[Math]]. - // If the option "spark.sql.dialect.spark.ansi.enabled" is enabled and there is corresponding + // If the option "spark.sql.ansi.enabled" is enabled and there is corresponding // function in [[Math]], the exact function will be called instead of evaluation with [[symbol]]. def exactMathMethod: Option[String] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index c19b7b978a01f..c96f2da9a5289 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -94,7 +94,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log lexer.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced lexer.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled lexer.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - lexer.SQL_standard_keyword_behavior = conf.dialectSparkAnsiEnabled + lexer.SQL_standard_keyword_behavior = conf.ansiEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -104,7 +104,7 @@ abstract class AbstractSqlParser(conf: SQLConf) extends ParserInterface with Log parser.legacy_setops_precedence_enbled = conf.setOpsPrecedenceEnforced parser.legacy_exponent_literal_as_decimal_enabled = conf.exponentLiteralAsDecimalEnabled parser.legacy_create_hive_table_by_default_enabled = conf.createHiveTableByDefaultEnabled - parser.SQL_standard_keyword_behavior = conf.dialectSparkAnsiEnabled + parser.SQL_standard_keyword_behavior = conf.ansiEnabled try { try { 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 743d88d7e457c..2e7c68a347192 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 @@ -1686,20 +1686,14 @@ object SQLConf { .checkValues(Dialect.values.map(_.toString)) .createWithDefault(Dialect.SPARK.toString) - val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = + buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() - .doc("This configuration is deprecated and will be removed in the future releases." + - "It is replaced by spark.sql.dialect.spark.ansi.enabled.") + .doc("When this option is set to true, creating managed tables with nonempty location " + + "is allowed. Otherwise, an analysis exception is thrown. ") .booleanConf .createWithDefault(false) - val DIALECT_SPARK_ANSI_ENABLED = buildConf("spark.sql.dialect.spark.ansi.enabled") - .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + - "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + - "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + - "the SQL parser.") - .fallbackConf(ANSI_ENABLED) - val VALIDATE_PARTITION_COLUMNS = buildConf("spark.sql.sources.validatePartitionColumns") .internal() @@ -1820,6 +1814,14 @@ object SQLConf { .checkValues(IntervalStyle.values.map(_.toString)) .createWithDefault(IntervalStyle.MULTI_UNITS.toString) + val ANSI_ENABLED = buildConf("spark.sql.ansi.enabled") + .doc("When true, Spark tries to conform to the ANSI SQL specification: 1. Spark will " + + "throw a runtime exception if an overflow occurs in any operation on integral/decimal " + + "field. 2. Spark will forbid using the reserved keywords of ANSI SQL as identifiers in " + + "the SQL parser.") + .booleanConf + .createWithDefault(false) + val SORT_BEFORE_REPARTITION = buildConf("spark.sql.execution.sortBeforeRepartition") .internal() @@ -2555,11 +2557,9 @@ class SQLConf extends Serializable with Logging { def intervalOutputStyle: IntervalStyle.Value = IntervalStyle.withName(getConf(INTERVAL_STYLE)) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString - - def dialectSparkAnsiEnabled: Boolean = getConf(DIALECT_SPARK_ANSI_ENABLED) + def ansiEnabled: Boolean = getConf(ANSI_ENABLED) - def ansiEnabled: Boolean = usePostgreSQLDialect || dialectSparkAnsiEnabled + def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString() def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index 62e688e4d4bd6..c1f1be3b30e4b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -436,7 +436,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testAndVerifyNotLeakingReflectionObjects( s"overflowing $testName, ansiEnabled=$ansiEnabled") { withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> ansiEnabled.toString + SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString ) { // Need to construct Encoder here rather than implicitly resolving it // so that SQLConf changes are respected. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index fe068f7a5f6c2..1a1cab823d4f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -169,7 +169,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } private def testDecimalOverflow(schema: StructType, row: Row): Unit = { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val encoder = RowEncoder(schema).resolveAndBind() intercept[Exception] { encoder.toRow(row) @@ -182,7 +182,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val encoder = RowEncoder(schema).resolveAndBind() assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 6e3fc438e41ea..ad8b1a1673679 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -61,7 +61,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Add(positiveLongLit, negativeLongLit), -1L) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Add, tpe, tpe) } @@ -80,7 +80,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(UnaryMinus(Literal(Int.MinValue)), Int.MinValue) checkEvaluation(UnaryMinus(Literal(Short.MinValue)), Short.MinValue) checkEvaluation(UnaryMinus(Literal(Byte.MinValue)), Byte.MinValue) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkExceptionInExpression[ArithmeticException]( UnaryMinus(Literal(Long.MinValue)), "overflow") checkExceptionInExpression[ArithmeticException]( @@ -122,7 +122,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Subtract(positiveLongLit, negativeLongLit), positiveLong - negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericAndInterval.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Subtract, tpe, tpe) } @@ -144,7 +144,7 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Multiply(positiveLongLit, negativeLongLit), positiveLong * negativeLong) Seq("true", "false").foreach { checkOverflow => - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> checkOverflow) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> checkOverflow) { DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => checkConsistencyBetweenInterpretedAndCodegenAllowingException(Multiply, tpe, tpe) } @@ -445,12 +445,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minLongLiteral, minLongLiteral) val e5 = Subtract(minLongLiteral, maxLongLiteral) val e6 = Multiply(minLongLiteral, minLongLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Long.MinValue) checkEvaluation(e2, Long.MinValue) checkEvaluation(e3, -2L) @@ -469,12 +469,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minIntLiteral, minIntLiteral) val e5 = Subtract(minIntLiteral, maxIntLiteral) val e6 = Multiply(minIntLiteral, minIntLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Int.MinValue) checkEvaluation(e2, Int.MinValue) checkEvaluation(e3, -2) @@ -493,12 +493,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minShortLiteral, minShortLiteral) val e5 = Subtract(minShortLiteral, maxShortLiteral) val e6 = Multiply(minShortLiteral, minShortLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Short.MinValue) checkEvaluation(e2, Short.MinValue) checkEvaluation(e3, (-2).toShort) @@ -517,12 +517,12 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val e4 = Add(minByteLiteral, minByteLiteral) val e5 = Subtract(minByteLiteral, maxByteLiteral) val e6 = Multiply(minByteLiteral, minByteLiteral) - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { Seq(e1, e2, e3, e4, e5, e6).foreach { e => checkExceptionInExpression[ArithmeticException](e, "overflow") } } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(e1, Byte.MinValue) checkEvaluation(e2, Byte.MinValue) checkEvaluation(e3, (-2).toByte) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index dde25ad994c5b..1490c114bb1d8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -894,8 +894,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to decimal type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { checkExceptionInExpression[ArithmeticException]( cast(Literal("134.12"), DecimalType(3, 2)), "cannot be represented") checkExceptionInExpression[ArithmeticException]( @@ -961,8 +960,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to byte type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ByteType) Seq(Byte.MaxValue + 1, Byte.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ByteType), "overflow") @@ -987,8 +985,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to short type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(ShortType) Seq(Short.MaxValue + 1, Short.MinValue - 1).foreach { value => checkExceptionInExpression[ArithmeticException](cast(value, ShortType), "overflow") @@ -1013,8 +1010,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to int type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key ->requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testIntMaxAndMin(IntegerType) testLongMaxAndMin(IntegerType) @@ -1031,8 +1027,7 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("Throw exception on casting out-of-range value to long type") { - withSQLConf( - SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { + withSQLConf(SQLConf.ANSI_ENABLED.key -> requiredAnsiEnabledForOverflowTestCases.toString) { testLongMaxAndMin(LongType) Seq(Long.MaxValue, 0, Long.MinValue).foreach { value => @@ -1209,7 +1204,7 @@ class CastSuite extends CastSuiteBase { } test("SPARK-28470: Cast should honor nullOnOverflow property") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(Cast(Literal("134.12"), DecimalType(3, 2)), null) checkEvaluation( Cast(Literal(Timestamp.valueOf("2019-07-25 22:04:36")), DecimalType(3, 2)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index 8609d888b7bc9..36bc3db580400 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -32,7 +32,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) @@ -41,7 +41,7 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { evaluateWithoutCodegen(overflowExpr, null) checkEvaluationWithUnsafeProjection(overflowExpr, null) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index cf6ebfb0ecefb..c5ffc381b58e2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -57,7 +57,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, @@ -69,7 +69,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(e2.getCause.isInstanceOf[ArithmeticException]) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val udf = ScalaUDF( (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), DecimalType.SYSTEM_DEFAULT, 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 0b694ea954156..90bb487ec6168 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 @@ -645,7 +645,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual(s"${sign}interval $intervalValue", expectedLiteral) // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } } @@ -732,12 +732,12 @@ class ExpressionParserSuite extends AnalysisTest { test("SPARK-23264 Interval Compatibility tests") { def checkIntervals(intervalValue: String, expected: Literal): Unit = { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual(intervalValue, expected) } // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { val aliases = defaultParser.parseExpression(intervalValue).collect { case a @ Alias(_: Literal, name) if intervalUnits.exists { unit => name.startsWith(unit.toString) } => a @@ -835,12 +835,12 @@ class ExpressionParserSuite extends AnalysisTest { } test("current date/timestamp braceless expressions") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "false") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 904ae384154c8..23063bbab7aa2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -660,7 +660,7 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { } test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { - withSQLConf(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key -> "true") { + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { reservedKeywordsInAnsiMode.foreach { keyword => val errMsg = intercept[ParseException] { parseTableIdentifier(keyword) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index a1fe95462ecae..7abf903bc6bee 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -46,9 +46,9 @@ select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); -- [SPARK-28036] Built-in udf left/right has inconsistent behavior -- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.dialect.spark.ansi.enabled=false; +set spark.sql.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.dialect.spark.ansi.enabled=true; +set spark.sql.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out index cac1e7ee5ab12..2e1d639974ec6 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -151,11 +151,11 @@ edcba -- !query 18 -set spark.sql.dialect.spark.ansi.enabled=false +set spark.sql.ansi.enabled=false -- !query 18 schema struct -- !query 18 output -spark.sql.dialect.spark.ansi.enabled false +spark.sql.ansi.enabled false -- !query 19 @@ -177,11 +177,11 @@ struct -- !query 20 -set spark.sql.dialect.spark.ansi.enabled=true +set spark.sql.ansi.enabled=true -- !query 20 schema struct -- !query 20 output -spark.sql.dialect.spark.ansi.enabled true +spark.sql.ansi.enabled true -- !query 21 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 13c2f9a810dcb..5269da1651e76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -164,7 +164,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession { DecimalData(BigDecimal("9"* 20 + ".123"), BigDecimal("9"* 20 + ".123")) :: Nil).toDF() Seq(true, false).foreach { ansiEnabled => - withSQLConf((SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, ansiEnabled.toString)) { + withSQLConf((SQLConf.ANSI_ENABLED.key, ansiEnabled.toString)) { val structDf = largeDecimals.select("a").agg(sum("a")) if (!ansiEnabled) { checkAnswer(structDf, Row(null)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index cd4b43f450c0e..ca0e734c2bcc4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -332,10 +332,10 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) - localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) case _: AnsiTest => - localSparkSession.conf.set(SQLConf.DIALECT_SPARK_ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index b564bb2d24005..98d69871c3429 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -118,13 +118,13 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") case _: AnsiTest => - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") case _ => statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") - statement.execute(s"SET ${SQLConf.DIALECT_SPARK_ANSI_ENABLED.key} = false") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") } // Run the SQL queries preparing them for comparison. From f0674e340d052a7d09d8be46ff1fc55ea2cb40d4 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 19:24:30 +0800 Subject: [PATCH 04/16] Revert "[SPARK-29364][SQL] Return an interval from date subtract according to SQL standard" This reverts commit d11cbf2e367317e6a2bedfcd71819ddf485c39c5. --- .../expressions/datetimeExpressions.scala | 22 -------------- .../sql/catalyst/util/DateTimeUtils.scala | 18 +---------- .../expressions/DateExpressionsSuite.scala | 30 ++++--------------- 3 files changed, 6 insertions(+), 64 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 b3aeda852a4dd..a36dc48017acd 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 @@ -2145,25 +2145,3 @@ case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expressi s"new org.apache.spark.unsafe.types.CalendarInterval(0, 0, $end - $start)") } } - -/** - * Returns the interval from the `left` date (inclusive) to the `right` date (exclusive). - */ -case class SubtractDates(left: Expression, right: Expression) - extends BinaryExpression with ImplicitCastInputTypes { - - override def inputTypes: Seq[AbstractDataType] = Seq(DateType, DateType) - override def dataType: DataType = CalendarIntervalType - - override def nullSafeEval(leftDays: Any, rightDays: Any): Any = { - DateTimeUtils.subtractDates(leftDays.asInstanceOf[Int], rightDays.asInstanceOf[Int]) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (leftDays, rightDays) => { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - s"$dtu.subtractDates($leftDays, $rightDays)" - }) - } -} - diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 6800abb2ae109..e5e25e251f128 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.types.UTF8String /** * Helper functions for converting between internal and external date and time representations. @@ -939,20 +939,4 @@ object DateTimeUtils { None } } - - /** - * Subtracts two dates. - * @param endDate - the end date, exclusive - * @param startDate - the start date, inclusive - * @return an interval between two dates. The interval can be negative - * if the end date is before the start date. - */ - def subtractDates(endDate: SQLDate, startDate: SQLDate): CalendarInterval = { - val period = Period.between( - LocalDate.ofEpochDay(startDate), - LocalDate.ofEpochDay(endDate)) - val months = period.getMonths + 12 * period.getYears - val days = period.getDays - new CalendarInterval(months, days, 0) - } } 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 7d015aa478df0..64d7ea5e8b791 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset} +import java.time.{Instant, LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -1103,39 +1103,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("timestamps difference") { val end = Instant.parse("2019-10-04T11:04:01.123456Z") - checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), + checkEvaluation(TimestampDiff(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) - checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), + checkEvaluation(TimestampDiff(Literal(end), Literal(Instant.EPOCH)), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds"))) - checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), + checkEvaluation(TimestampDiff(Literal(Instant.EPOCH), Literal(end)), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds"))) checkEvaluation( - SubtractTimestamps( + TimestampDiff( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds"))) } - - test("subtract dates") { - val end = LocalDate.of(2019, 10, 5) - checkEvaluation(SubtractDates(Literal(end), Literal(end)), - new CalendarInterval(0, 0, 0)) - checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), - IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 days"))) - checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), - IntervalUtils.stringToInterval(UTF8String.fromString("interval -1 days"))) - val epochDate = Literal(LocalDate.ofEpochDay(0)) - checkEvaluation(SubtractDates(Literal(end), epochDate), - IntervalUtils.stringToInterval(UTF8String.fromString("interval 49 years 9 months 4 days"))) - checkEvaluation(SubtractDates(epochDate, Literal(end)), - IntervalUtils.stringToInterval(UTF8String.fromString("interval -49 years -9 months -4 days"))) - checkEvaluation( - SubtractDates( - Literal(LocalDate.of(10000, 1, 1)), - Literal(LocalDate.of(1, 1, 1))), - IntervalUtils.stringToInterval(UTF8String.fromString("interval 9999 years"))) - } } From 8768937c7e13e41b4d74ad96983ecdfa2ec0d0e6 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 19:29:42 +0800 Subject: [PATCH 05/16] Revert "[SPARK-28997][SQL] Add `spark.sql.dialect`" This reverts commit a1213d5f963f6e8815bbfaff308b0a24112fff54. --- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../catalyst/analysis/PostgreSQLDialect.scala | 49 ------------ .../sql/catalyst/analysis/TypeCoercion.scala | 3 +- .../PostgreCastStringToBoolean.scala | 80 ------------------- .../spark/sql/catalyst/util/StringUtils.scala | 5 +- .../util/postgreSQL/StringUtils.scala | 33 -------- .../apache/spark/sql/internal/SQLConf.scala | 26 ++---- .../catalyst/analysis/TypeCoercionSuite.scala | 8 +- .../sql/catalyst/expressions/CastSuite.scala | 24 ++++-- .../expressions/postgreSQL/CastSuite.scala | 59 -------------- .../sql/PostgreSQLDialectQuerySuite.scala | 42 ---------- .../apache/spark/sql/SQLQueryTestSuite.scala | 3 +- .../ThriftServerQueryTestSuite.scala | 5 +- 13 files changed, 37 insertions(+), 307 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala 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 659e4a5c86ec1..e292514667500 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 @@ -232,7 +232,6 @@ class Analyzer( ResolveBinaryArithmetic(conf) :: TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), - Batch("PostgreSQL Dialect", Once, PostgreSQLDialect.postgreSQLDialectRules: _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), Batch("Remove Unresolved Hints", Once, new ResolveHints.RemoveAllHints(conf)), @@ -287,11 +286,7 @@ class Analyzer( case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) case (TimestampType, _) => SubtractTimestamps(l, r) case (_, TimestampType) => SubtractTimestamps(l, r) - case (_, DateType) => if (conf.usePostgreSQLDialect) { - DateDiff(l, r) - } else { - SubtractDates(l, r) - } + case (_, DateType) => SubtractDates(l, r) case (DateType, _) => DateSub(l, r) case _ => s } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala deleted file mode 100644 index 934e53703e241..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/PostgreSQLDialect.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.analysis - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.Cast -import org.apache.spark.sql.catalyst.expressions.postgreSQL.PostgreCastStringToBoolean -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, StringType} - -object PostgreSQLDialect { - val postgreSQLDialectRules: List[Rule[LogicalPlan]] = - CastStringToBoolean :: - Nil - - object CastStringToBoolean extends Rule[LogicalPlan] with Logging { - override def apply(plan: LogicalPlan): LogicalPlan = { - // The SQL configuration `spark.sql.dialect` can be changed in runtime. - // To make sure the configuration is effective, we have to check it during rule execution. - val conf = SQLConf.get - if (conf.usePostgreSQLDialect) { - plan.transformExpressions { - case Cast(child, dataType, _) - if dataType == BooleanType && child.dataType == StringType => - PostgreCastStringToBoolean(child) - } - } else { - plan - } - } - } -} 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 e76193fd94222..587cc7607c040 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 @@ -673,9 +673,8 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - val preferIntegralDivision = conf.usePostgreSQLDialect (left.dataType, right.dataType) match { - case (_: IntegralType, _: IntegralType) if preferIntegralDivision => + case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => IntegralDivide(left, right) case _ => Divide(Cast(left, DoubleType), Cast(right, DoubleType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala deleted file mode 100644 index 0e87707d01e47..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastStringToBoolean.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.catalyst.expressions.postgreSQL - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant, UnaryExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, JavaCode} -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils -import org.apache.spark.sql.types.{BooleanType, DataType, StringType} -import org.apache.spark.unsafe.types.UTF8String - -case class PostgreCastStringToBoolean(child: Expression) - extends UnaryExpression with NullIntolerant { - - override def checkInputDataTypes(): TypeCheckResult = { - if (child.dataType == StringType) { - TypeCheckResult.TypeCheckSuccess - } else { - TypeCheckResult.TypeCheckFailure( - s"The expression ${getClass.getSimpleName} only accepts string input data type") - } - } - - override def nullSafeEval(input: Any): Any = { - val s = input.asInstanceOf[UTF8String].trim().toLowerCase() - if (StringUtils.isTrueString(s)) { - true - } else if (StringUtils.isFalseString(s)) { - false - } else { - null - } - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" - val eval = child.genCode(ctx) - val javaType = JavaCode.javaType(dataType) - val preprocessedString = ctx.freshName("preprocessedString") - val castCode = - code""" - boolean ${ev.isNull} = ${eval.isNull}; - $javaType ${ev.value} = false; - if (!${eval.isNull}) { - UTF8String $preprocessedString = ${eval.value}.trim().toLowerCase(); - if ($stringUtils.isTrueString($preprocessedString)) { - ${ev.value} = true; - } else if ($stringUtils.isFalseString($preprocessedString)) { - ${ev.value} = false; - } else { - ${ev.isNull} = true; - } - } - """ - ev.copy(code = eval.code + castCode) - } - - override def dataType: DataType = BooleanType - - override def nullable: Boolean = true - - override def toString: String = s"PostgreCastStringToBoolean($child as ${dataType.simpleString})" - - override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 51ab3df0e0bfb..f6af45dac5c6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -66,11 +66,12 @@ object StringUtils extends Logging { "(?s)" + out.result() // (?s) enables dotall mode, causing "." to match new lines } + // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. private[this] val trueStrings = - Set("t", "true", "y", "yes", "1").map(UTF8String.fromString) + Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) private[this] val falseStrings = - Set("f", "false", "n", "no", "0").map(UTF8String.fromString) + Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) // scalastyle:off caselocale def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.trimAll().toLowerCase) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala deleted file mode 100644 index 1ae15df29d6e7..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/postgreSQL/StringUtils.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.util.postgreSQL - -import org.apache.spark.unsafe.types.UTF8String - -object StringUtils { - // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. - private[this] val trueStrings = - Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) - - private[this] val falseStrings = - Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) - - def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s) - - def isFalseString(s: UTF8String): Boolean = falseStrings.contains(s) -} 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 2e7c68a347192..589ef933fa72c 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 @@ -1669,22 +1669,12 @@ object SQLConf { .booleanConf .createWithDefault(false) - object Dialect extends Enumeration { - val SPARK, POSTGRESQL = Value - } - - val DIALECT = - buildConf("spark.sql.dialect") - .doc("The specific features of the SQL language to be adopted, which are available when " + - "accessing the given database. Currently, Spark supports two database dialects, `Spark` " + - "and `PostgreSQL`. With `PostgreSQL` dialect, Spark will: " + - "1. perform integral division with the / operator if both sides are integral types; " + - "2. accept \"true\", \"yes\", \"1\", \"false\", \"no\", \"0\", and unique prefixes as " + - "input and trim input for the boolean data type.") - .stringConf - .transform(_.toUpperCase(Locale.ROOT)) - .checkValues(Dialect.values.map(_.toString)) - .createWithDefault(Dialect.SPARK.toString) + val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") + .internal() + .doc("When true, will perform integral division with the / operator " + + "if both sides are integral types. This is for PostgreSQL test cases only.") + .booleanConf + .createWithDefault(false) val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") @@ -2547,6 +2537,8 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) + def preferIntegralDivision: Boolean = getConf(PREFER_INTEGRAL_DIVISION) + def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) def partitionOverwriteMode: PartitionOverwriteMode.Value = @@ -2559,8 +2551,6 @@ class SQLConf extends Serializable with Logging { def ansiEnabled: Boolean = getConf(ANSI_ENABLED) - def usePostgreSQLDialect: Boolean = getConf(DIALECT) == Dialect.POSTGRESQL.toString() - def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) def serializerNestedSchemaPruningEnabled: Boolean = 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 f35617b374c99..02b812fd4792c 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 @@ -1452,15 +1452,15 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-28395 Division operator support integral division") { val rules = Seq(FunctionArgumentConversion, Division(conf)) - Seq(SQLConf.Dialect.SPARK, SQLConf.Dialect.POSTGRESQL).foreach { dialect => - withSQLConf(SQLConf.DIALECT.key -> dialect.toString) { - val result1 = if (dialect == SQLConf.Dialect.POSTGRESQL) { + Seq(true, false).foreach { preferIntegralDivision => + withSQLConf(SQLConf.PREFER_INTEGRAL_DIVISION.key -> s"$preferIntegralDivision") { + val result1 = if (preferIntegralDivision) { IntegralDivide(1L, 1L) } else { Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) } ruleTest(rules, Divide(1L, 1L), result1) - val result2 = if (dialect == SQLConf.Dialect.POSTGRESQL) { + val result2 = if (preferIntegralDivision) { IntegralDivide(1, Cast(1, ShortType)) } else { Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 1490c114bb1d8..3193df1ba7554 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -679,25 +679,37 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to boolean") { - checkCast("t", true) checkCast("true", true) + checkCast("tru", true) + checkCast("tr", true) + checkCast("t", true) checkCast("tRUe", true) - checkCast("y", true) + checkCast(" tRue ", true) + checkCast(" tRu ", true) checkCast("yes", true) + checkCast("ye", true) + checkCast("y", true) checkCast("1", true) checkCast("1 ", true) checkCast("f", false) checkCast("f\t", false) checkCast("false", false) - checkCast("FAlsE", false) - checkCast("n", false) + checkCast("fals", false) + checkCast("fal", false) + checkCast("fa", false) + checkCast("f", false) + checkCast(" fAlse ", false) + checkCast(" fAls ", false) + checkCast(" FAlsE ", false) checkCast("no", false) + checkCast("n", false) checkCast("0", false) + checkCast("off", false) + checkCast("of", false) + checkEvaluation(cast("o", BooleanType), null) checkEvaluation(cast("abc", BooleanType), null) - checkEvaluation(cast("tru", BooleanType), null) - checkEvaluation(cast("fla", BooleanType), null) checkEvaluation(cast("", BooleanType), null) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala deleted file mode 100644 index 175904da21969..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/CastSuite.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.catalyst.expressions.postgreSQL - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal} - -class CastSuite extends SparkFunSuite with ExpressionEvalHelper { - private def checkPostgreCastStringToBoolean(v: Any, expected: Any): Unit = { - checkEvaluation(PostgreCastStringToBoolean(Literal(v)), expected) - } - - test("cast string to boolean") { - checkPostgreCastStringToBoolean("true", true) - checkPostgreCastStringToBoolean("tru", true) - checkPostgreCastStringToBoolean("tr", true) - checkPostgreCastStringToBoolean("t", true) - checkPostgreCastStringToBoolean("tRUe", true) - checkPostgreCastStringToBoolean(" tRue ", true) - checkPostgreCastStringToBoolean(" tRu ", true) - checkPostgreCastStringToBoolean("yes", true) - checkPostgreCastStringToBoolean("ye", true) - checkPostgreCastStringToBoolean("y", true) - checkPostgreCastStringToBoolean("1", true) - checkPostgreCastStringToBoolean("on", true) - - checkPostgreCastStringToBoolean("false", false) - checkPostgreCastStringToBoolean("fals", false) - checkPostgreCastStringToBoolean("fal", false) - checkPostgreCastStringToBoolean("fa", false) - checkPostgreCastStringToBoolean("f", false) - checkPostgreCastStringToBoolean(" fAlse ", false) - checkPostgreCastStringToBoolean(" fAls ", false) - checkPostgreCastStringToBoolean(" FAlsE ", false) - checkPostgreCastStringToBoolean("no", false) - checkPostgreCastStringToBoolean("n", false) - checkPostgreCastStringToBoolean("0", false) - checkPostgreCastStringToBoolean("off", false) - checkPostgreCastStringToBoolean("of", false) - - checkPostgreCastStringToBoolean("o", null) - checkPostgreCastStringToBoolean("abc", null) - checkPostgreCastStringToBoolean("", null) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala deleted file mode 100644 index 1354dcfda45fe..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/PostgreSQLDialectQuerySuite.scala +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql - -import org.apache.spark.SparkConf -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSparkSession - -class PostgreSQLDialectQuerySuite extends QueryTest with SharedSparkSession { - - override def sparkConf: SparkConf = - super.sparkConf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) - - test("cast string to boolean") { - Seq("true", "tru", "tr", "t", " tRue ", " tRu ", "yes", "ye", - "y", "1", "on").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(true)) - } - Seq("false", "fals", "fal", "fa", "f", " fAlse ", " fAls ", "no", "n", - "0", "off", "of").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(false)) - } - - Seq("o", "abc", "").foreach { input => - checkAnswer(sql(s"select cast('$input' as boolean)"), Row(null)) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index ca0e734c2bcc4..7c86cdcf0284d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -333,8 +333,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) - localSparkSession.conf.set(SQLConf.DIALECT.key, SQLConf.Dialect.POSTGRESQL.toString) - case _: AnsiTest => + localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 98d69871c3429..cbd36a6fce60d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -119,11 +119,8 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.POSTGRESQL.toString}") - case _: AnsiTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") case _ => - statement.execute(s"SET ${SQLConf.DIALECT.key} = ${SQLConf.Dialect.SPARK.toString}") statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") } From 94da433002b767fa2e0b548cc247072487dc2cc9 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 19:30:35 +0800 Subject: [PATCH 06/16] Revert "[SPARK-27931][SQL] Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type" This reverts commit 3b07a4eb288ef1ab060d1fbed059a31e9da92609. --- .../spark/sql/catalyst/util/StringUtils.scala | 8 ++---- .../sql/catalyst/expressions/CastSuite.scala | 25 +++---------------- .../sql-tests/inputs/postgreSQL/boolean.sql | 5 +++- .../results/postgreSQL/boolean.sql.out | 10 ++++---- 4 files changed, 15 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index f6af45dac5c6d..76fe682ef7ccf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -66,12 +66,8 @@ object StringUtils extends Logging { "(?s)" + out.result() // (?s) enables dotall mode, causing "." to match new lines } - // "true", "yes", "1", "false", "no", "0", and unique prefixes of these strings are accepted. - private[this] val trueStrings = - Set("true", "tru", "tr", "t", "yes", "ye", "y", "on", "1").map(UTF8String.fromString) - - private[this] val falseStrings = - Set("false", "fals", "fal", "fa", "f", "no", "n", "off", "of", "0").map(UTF8String.fromString) + private[this] val trueStrings = Set("t", "true", "y", "yes", "1").map(UTF8String.fromString) + private[this] val falseStrings = Set("f", "false", "n", "no", "0").map(UTF8String.fromString) // scalastyle:off caselocale def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.trimAll().toLowerCase) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 3193df1ba7554..2d8f22c34ade7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -679,36 +679,19 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to boolean") { - checkCast("true", true) - checkCast("tru", true) - checkCast("tr", true) checkCast("t", true) + checkCast("true", true) checkCast("tRUe", true) - checkCast(" tRue ", true) - checkCast(" tRu ", true) - checkCast("yes", true) - checkCast("ye", true) checkCast("y", true) + checkCast("yes", true) checkCast("1", true) - checkCast("1 ", true) - checkCast("f", false) - checkCast("f\t", false) checkCast("false", false) - checkCast("fals", false) - checkCast("fal", false) - checkCast("fa", false) - checkCast("f", false) - checkCast(" fAlse ", false) - checkCast(" fAls ", false) - checkCast(" FAlsE ", false) - checkCast("no", false) + checkCast("FAlsE", false) checkCast("n", false) + checkCast("no", false) checkCast("0", false) - checkCast("off", false) - checkCast("of", false) - checkEvaluation(cast("o", BooleanType), null) checkEvaluation(cast("abc", BooleanType), null) checkEvaluation(cast("", BooleanType), null) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql index 178823bcfe9d6..4427d76f48d80 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql @@ -22,6 +22,7 @@ SELECT false AS `false`; SELECT boolean('t') AS true; +-- [SPARK-27931] Trim the string when cast string type to boolean type SELECT boolean(' f ') AS `false`; SELECT boolean('true') AS true; @@ -48,10 +49,12 @@ SELECT boolean('no') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL SELECT boolean('nay') AS error; +-- [SPARK-27931] Accept 'on' and 'off' as input for boolean data type SELECT boolean('on') AS true; SELECT boolean('off') AS `false`; +-- [SPARK-27931] Accept unique prefixes thereof SELECT boolean('of') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL @@ -98,7 +101,7 @@ SELECT boolean('f') <= boolean('t') AS true; -- explicit casts to/from text SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; - +-- [SPARK-27931] Trim the string when cast to boolean type SELECT boolean(string(' true ')) AS true, boolean(string(' FALSE')) AS `false`; SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index 203806d43368a..c7903c8a34ef4 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -39,7 +39,7 @@ SELECT boolean(' f ') AS `false` -- !query 4 schema struct -- !query 4 output -false +NULL -- !query 5 @@ -127,7 +127,7 @@ SELECT boolean('on') AS true -- !query 15 schema struct -- !query 15 output -true +NULL -- !query 16 @@ -135,7 +135,7 @@ SELECT boolean('off') AS `false` -- !query 16 schema struct -- !query 16 output -false +NULL -- !query 17 @@ -143,7 +143,7 @@ SELECT boolean('of') AS `false` -- !query 17 schema struct -- !query 17 output -false +NULL -- !query 18 @@ -296,7 +296,7 @@ SELECT boolean(string(' true ')) AS true, -- !query 36 schema struct -- !query 36 output -true false +NULL NULL -- !query 37 From 9638331d303b9b3292fc0a2d5a82ecdf3c9322c5 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 3 Dec 2019 19:31:43 +0800 Subject: [PATCH 07/16] Revert "[SPARK-28395][SQL] Division operator support integral division" This reverts commit 69268492471137dd7a3da54c218026c3b1fa7db3. --- .../sql/catalyst/analysis/TypeCoercion.scala | 11 +++----- .../apache/spark/sql/internal/SQLConf.scala | 17 ------------ .../catalyst/analysis/TypeCoercionSuite.scala | 27 ++----------------- 3 files changed, 5 insertions(+), 50 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 587cc7607c040..dd174ef28f309 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 @@ -59,7 +59,7 @@ object TypeCoercion { CaseWhenCoercion :: IfCoercion :: StackCoercion :: - Division(conf) :: + Division :: ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: @@ -662,7 +662,7 @@ object TypeCoercion { * Hive only performs integral division with the DIV operator. The arguments to / are always * converted to fractional types. */ - case class Division(conf: SQLConf) extends TypeCoercionRule { + object Division extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, @@ -673,12 +673,7 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - (left.dataType, right.dataType) match { - case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => - IntegralDivide(left, right) - case _ => - Divide(Cast(left, DoubleType), Cast(right, DoubleType)) - } + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) } private def isNumericOrNull(ex: Expression): Boolean = { 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 589ef933fa72c..8024c5e3377c6 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 @@ -1669,21 +1669,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") - .internal() - .doc("When true, will perform integral division with the / operator " + - "if both sides are integral types. This is for PostgreSQL test cases only.") - .booleanConf - .createWithDefault(false) - - val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = - buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") - .internal() - .doc("When this option is set to true, creating managed tables with nonempty location " + - "is allowed. Otherwise, an analysis exception is thrown. ") - .booleanConf - .createWithDefault(false) - val VALIDATE_PARTITION_COLUMNS = buildConf("spark.sql.sources.validatePartitionColumns") .internal() @@ -2537,8 +2522,6 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) - def preferIntegralDivision: Boolean = getConf(PREFER_INTEGRAL_DIVISION) - def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) def partitionOverwriteMode: PartitionOverwriteMode.Value = 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 02b812fd4792c..0d6f9bcedb6a2 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 @@ -1425,7 +1425,7 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + "in aggregation function like sum") { - val rules = Seq(FunctionArgumentConversion, Division(conf)) + val rules = Seq(FunctionArgumentConversion, Division) // Casts Integer to Double ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will @@ -1444,35 +1444,12 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division(conf), ImplicitTypeCasts) + val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } - test("SPARK-28395 Division operator support integral division") { - val rules = Seq(FunctionArgumentConversion, Division(conf)) - Seq(true, false).foreach { preferIntegralDivision => - withSQLConf(SQLConf.PREFER_INTEGRAL_DIVISION.key -> s"$preferIntegralDivision") { - val result1 = if (preferIntegralDivision) { - IntegralDivide(1L, 1L) - } else { - Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) - } - ruleTest(rules, Divide(1L, 1L), result1) - val result2 = if (preferIntegralDivision) { - IntegralDivide(1, Cast(1, ShortType)) - } else { - Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) - } - ruleTest(rules, Divide(1, Cast(1, ShortType)), result2) - - ruleTest(rules, Divide(1L, 1D), Divide(Cast(1L, DoubleType), Cast(1D, DoubleType))) - ruleTest(rules, Divide(Decimal(1.1), 1L), Divide(Decimal(1.1), 1L)) - } - } - } - test("binary comparison with string promotion") { val rule = TypeCoercion.PromoteStrings(conf) ruleTest(rule, From c0da746fa2077c6bfb5bb529d2a4438a2c720a2f Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 4 Dec 2019 04:36:36 +0800 Subject: [PATCH 08/16] fix --- .../postgreSQL/PostgreCastToBoolean.scala | 83 ------------------- .../sql-tests/results/ansi/interval.sql.out | 4 +- .../sql-tests/results/interval.sql.out | 4 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 2 - .../ThriftServerQueryTestSuite.scala | 1 - 5 files changed, 4 insertions(+), 90 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala deleted file mode 100644 index 02bc6f0d0d8bf..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/postgreSQL/PostgreCastToBoolean.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.catalyst.expressions.postgreSQL - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.{CastBase, Expression, TimeZoneAwareExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.postgreSQL.StringUtils -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -case class PostgreCastToBoolean(child: Expression, timeZoneId: Option[String]) - extends CastBase { - - override protected def ansiEnabled = - throw new UnsupportedOperationException("PostgreSQL dialect doesn't support ansi mode") - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override def checkInputDataTypes(): TypeCheckResult = child.dataType match { - case StringType | IntegerType | NullType => - TypeCheckResult.TypeCheckSuccess - case _ => - TypeCheckResult.TypeCheckFailure(s"cannot cast type ${child.dataType} to boolean") - } - - override def castToBoolean(from: DataType): Any => Any = from match { - case StringType => - buildCast[UTF8String](_, str => { - val s = str.trimAll().toLowerCase() - if (StringUtils.isTrueString(s)) { - true - } else if (StringUtils.isFalseString(s)) { - false - } else { - throw new IllegalArgumentException(s"invalid input syntax for type boolean: $s") - } - }) - case IntegerType => - super.castToBoolean(from) - } - - override def castToBooleanCode(from: DataType): CastFunction = from match { - case StringType => - val stringUtils = inline"${StringUtils.getClass.getName.stripSuffix("$")}" - (c, evPrim, evNull) => - code""" - if ($stringUtils.isTrueString($c.trim().toLowerCase())) { - $evPrim = true; - } else if ($stringUtils.isFalseString($c.trim().toLowerCase())) { - $evPrim = false; - } else { - throw new IllegalArgumentException("invalid input syntax for type boolean: $c"); - } - """ - - case IntegerType => - super.castToBooleanCode(from) - } - - override def dataType: DataType = BooleanType - - override def nullable: Boolean = child.nullable - - override def toString: String = s"PostgreCastToBoolean($child as ${dataType.simpleString})" - - override def sql: String = s"CAST(${child.sql} AS ${dataType.sql})" -} 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 2709fb42488cc..7d94711ded343 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 @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 601521ba43227..04e1d8dd95848 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 7c86cdcf0284d..53423a077f39f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -333,8 +333,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) - localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index cbd36a6fce60d..7b208a3b0233f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -119,7 +119,6 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { testCase match { case _: PgSQLTest => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") - statement.execute(s"SET ${SQLConf.PREFER_INTEGRAL_DIVISION.key} = true") case _ => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") } From f90f3b191c565d00d8014c3735a2bd95868619ae Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 4 Dec 2019 04:39:59 +0800 Subject: [PATCH 09/16] Revert "[SPARK-28343][FOLLOW-UP][SQL][TEST] Enable spark.sql.function.preferIntegralDivision for PostgreSQL testing" This reverts commit 71882f119e72934a02d4c177f0d52c785e2df79f. --- .../sql-tests/inputs/postgreSQL/int2.sql | 6 +- .../sql-tests/inputs/postgreSQL/int4.sql | 1 + .../sql-tests/inputs/postgreSQL/int8.sql | 1 + .../sql-tests/results/postgreSQL/case.sql.out | 18 ++--- .../sql-tests/results/postgreSQL/int2.sql.out | 4 +- .../sql-tests/results/postgreSQL/int4.sql.out | 32 ++++---- .../sql-tests/results/postgreSQL/int8.sql.out | 78 +++++++++---------- .../results/udf/postgreSQL/udf-case.sql.out | 8 +- 8 files changed, 76 insertions(+), 72 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql index 07f5976ca6d2f..835e02c36436a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql @@ -92,9 +92,11 @@ WHERE f1 > -32767; SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; -SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; +-- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. +SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i; -SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; +-- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. +SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i; -- corner cases SELECT string(shiftleft(smallint(-1), 15)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql index 3a409eea34837..8df3a0e698cbe 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql @@ -138,6 +138,7 @@ SELECT int('1000') < int('999') AS `false`; SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; +-- [SPARK-2659] HiveQL: Division operator should always perform fractional division SELECT 2 + 2 / 2 AS three; SELECT (2 + 2) / 2 AS two; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql index 5fea758e73084..ed46ae30ad283 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql @@ -87,6 +87,7 @@ SELECT 37 - q1 AS minus4 FROM INT8_TBL; SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; +-- [SPARK-2659] HiveQL: Division operator should always perform fractional division -- int8 op int4 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; -- int4 op int8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out index 348198b060238..a3410684e85dd 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out @@ -176,28 +176,28 @@ struct -- !query 18 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 18 schema -struct +struct -- !query 18 output -1 +1.0 -- !query 19 SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output -1 +1.0 -- !query 20 SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl -- !query 20 schema -struct 100) THEN (1 div 0) ELSE 0 END:int> +struct 100) THEN (CAST(1 AS DOUBLE) / CAST(0 AS DOUBLE)) ELSE CAST(0 AS DOUBLE) END:double> -- !query 20 output -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 -- !query 21 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out index d0a14618a5163..d7a3fdf481418 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -266,7 +266,7 @@ struct -- !query 27 -SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i -- !query 27 schema struct -- !query 27 output @@ -278,7 +278,7 @@ struct -- !query 28 -SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i -- !query 28 schema struct -- !query 28 output diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out index 30afd6e695842..02be07dda5df0 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -321,25 +321,25 @@ struct -- !query 33 SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema -struct +struct -- !query 33 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 +-123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 -- !query 34 SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema -struct +struct -- !query 34 output - -123456 -61728 - -2147483647 -1073741823 - 0 0 - 123456 61728 - 2147483647 1073741823 +-123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 -- !query 35 @@ -417,17 +417,17 @@ struct -- !query 44 SELECT 2 + 2 / 2 AS three -- !query 44 schema -struct +struct -- !query 44 output -3 +3.0 -- !query 45 SELECT (2 + 2) / 2 AS two -- !query 45 schema -struct +struct -- !query 45 output -2 +2.0 -- !query 46 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out index 7a8e706cb8d85..65306a5d0db45 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -409,13 +409,13 @@ struct -- !query 42 SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL -- !query 42 schema -struct +struct -- !query 42 output - 123 456 0 123 - 123 4567890123456789 0 123 - 4567890123456789 -4567890123456789 -1 0 - 4567890123456789 123 37137318076884 57 - 4567890123456789 4567890123456789 1 0 +123 456 0.26973684210526316 123 + 123 4567890123456789 2.6927092525360204E-14 123 + 4567890123456789 -4567890123456789 -1.0 0 + 4567890123456789 123 3.713731807688446E13 57 + 4567890123456789 4567890123456789 1.0 0 -- !query 43 @@ -493,49 +493,49 @@ struct -- !query 49 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL -- !query 49 schema -struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:bigint> +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> -- !query 49 output -165 81 5166 2 -165 81 5166 2 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -- !query 50 SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL -- !query 50 schema -struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:bigint> +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> -- !query 50 output -369 123 30258 2 -369 123 30258 2 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -- !query 51 SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL -- !query 51 schema -struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:bigint> +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> -- !query 51 output -165 81 5166 2 -165 81 5166 2 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 -4567890123456831 4567890123456747 191851385185185138 108759288653733 +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 -- !query 52 SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL -- !query 52 schema -struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:bigint> +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> -- !query 52 output -369 123 30258 2 -369 123 30258 2 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 -4567890123457035 -4567890123456543 1123700970370370094 0 +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 -- !query 53 @@ -569,7 +569,7 @@ struct -- !query 56 select bigint('9223372036854775800') / bigint('0') -- !query 56 schema -struct<(CAST(9223372036854775800 AS BIGINT) div CAST(0 AS BIGINT)):bigint> +struct<(CAST(CAST(9223372036854775800 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> -- !query 56 output NULL @@ -577,7 +577,7 @@ NULL -- !query 57 select bigint('-9223372036854775808') / smallint('0') -- !query 57 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(0 AS SMALLINT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(0 AS SMALLINT) AS DOUBLE)):double> -- !query 57 output NULL @@ -585,7 +585,7 @@ NULL -- !query 58 select smallint('100') / bigint('0') -- !query 58 schema -struct<(CAST(CAST(100 AS SMALLINT) AS BIGINT) div CAST(0 AS BIGINT)):bigint> +struct<(CAST(CAST(100 AS SMALLINT) AS DOUBLE) / CAST(CAST(0 AS BIGINT) AS DOUBLE)):double> -- !query 58 output NULL @@ -740,9 +740,9 @@ long overflow -- !query 74 SELECT bigint((-9223372036854775808)) / bigint((-1)) -- !query 74 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(-1 AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS BIGINT) AS DOUBLE)):double> -- !query 74 output --9223372036854775808 +9.223372036854776E18 -- !query 75 @@ -765,9 +765,9 @@ long overflow -- !query 77 SELECT bigint((-9223372036854775808)) / int((-1)) -- !query 77 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS INT) AS DOUBLE)):double> -- !query 77 output --9223372036854775808 +9.223372036854776E18 -- !query 78 @@ -790,9 +790,9 @@ long overflow -- !query 80 SELECT bigint((-9223372036854775808)) / smallint((-1)) -- !query 80 schema -struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +struct<(CAST(CAST(-9223372036854775808 AS BIGINT) AS DOUBLE) / CAST(CAST(-1 AS SMALLINT) AS DOUBLE)):double> -- !query 80 output --9223372036854775808 +9.223372036854776E18 -- !query 81 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out index 44a764ce4e6dd..79a45823da512 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out @@ -176,15 +176,15 @@ struct -- !query 18 SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 18 schema -struct +struct -- !query 18 output -1 +1.0 -- !query 19 SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END -- !query 19 schema -struct +struct -- !query 19 output 1 @@ -192,7 +192,7 @@ struct 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 20 schema -struct 100) THEN CAST(udf(cast((1 div 0) as string)) AS INT) ELSE CAST(udf(cast(0 as string)) AS INT) END:int> +struct 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string> -- !query 20 output 0 0 From 8e7e88797a8d9571da9d5336f51a57b515b53123 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 4 Dec 2019 20:26:13 +0800 Subject: [PATCH 10/16] fix SQLQueryTestSuite for the revert, mainly related with int div int --- .../sql-tests/results/datetime.sql.out | 0 .../sql-tests/results/postgreSQL/int4.sql.out | 4 +- .../sql-tests/results/postgreSQL/int8.sql.out | 2 +- .../postgreSQL/select_implicit.sql.out | 55 +++++++++++-------- .../results/postgreSQL/window_part1.sql.out | 2 +- .../results/udf/postgreSQL/udf-case.sql.out | 16 +++--- .../postgreSQL/udf-select_implicit.sql.out | 55 +++++++++++-------- .../apache/spark/sql/SQLQueryTestSuite.scala | 2 + 8 files changed, 78 insertions(+), 58 deletions(-) mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/datetime.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out old mode 100644 new mode 100755 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out old mode 100644 new mode 100755 index 02be07dda5df0..e1cb96d09ec75 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out @@ -323,7 +323,7 @@ SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i -- !query 33 schema struct -- !query 33 output --123456 -61728.0 + -123456 -61728.0 -2147483647 -1.0737418235E9 0 0.0 123456 61728.0 @@ -335,7 +335,7 @@ SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i -- !query 34 schema struct -- !query 34 output --123456 -61728.0 + -123456 -61728.0 -2147483647 -1.0737418235E9 0 0.0 123456 61728.0 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out old mode 100644 new mode 100755 index 65306a5d0db45..8e2700caa3345 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out @@ -411,7 +411,7 @@ SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL -- !query 42 schema struct -- !query 42 output -123 456 0.26973684210526316 123 + 123 456 0.26973684210526316 123 123 4567890123456789 2.6927092525360204E-14 123 4567890123456789 -4567890123456789 -1.0 0 4567890123456789 123 3.713731807688446E13 57 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out old mode 100644 new mode 100755 index 0675820b381da..e9ba62801d6a2 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -239,31 +239,36 @@ struct SELECT a/2, a/2 FROM test_missing_target ORDER BY a/2 -- !query 23 schema -struct<(a div 2):int,(a div 2):int> +struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> -- !query 23 output -0 0 -0 0 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 24 SELECT a/2, a/2 FROM test_missing_target GROUP BY a/2 ORDER BY a/2 -- !query 24 schema -struct<(a div 2):int,(a div 2):int> +struct<(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double,(CAST(a AS DOUBLE) / CAST(2 AS DOUBLE)):double> -- !query 24 output -0 0 -1 1 -2 2 -3 3 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 25 @@ -331,7 +336,8 @@ SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 struct -- !query 30 output 1 -5 +2 +3 4 @@ -370,8 +376,10 @@ SELECT count(b) FROM test_missing_target -- !query 33 schema struct -- !query 33 output -7 +4 3 +2 +1 -- !query 34 @@ -390,11 +398,12 @@ SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y WHERE x.a = y.a GROUP BY x.b/2 ORDER BY x.b/2 -- !query 35 schema -struct<(b div 2):int,count(b):bigint> +struct<(CAST(b AS DOUBLE) / CAST(2 AS DOUBLE)):double,count(b):bigint> -- !query 35 output -0 1 -1 5 -2 4 +0.5 1 +1.0 2 +1.5 3 +2.0 4 -- !query 36 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out old mode 100644 new mode 100755 index 45bc98ae97640..61b86b556e1a8 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part1.sql.out @@ -387,7 +387,7 @@ struct -- !query 23 SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 -- !query 23 schema -struct +struct -- !query 23 output 0.0 0.0 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out old mode 100644 new mode 100755 index 79a45823da512..7012b15213662 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out @@ -176,7 +176,7 @@ struct -- !query 18 SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END -- !query 18 schema -struct +struct -- !query 18 output 1.0 @@ -184,20 +184,20 @@ struct +struct -- !query 19 output -1 +1.0 -- !query 20 SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl -- !query 20 schema -struct 100) THEN udf((cast(1 as double) / cast(0 as double))) ELSE udf(0) END:string> +struct 100) THEN CAST(udf(cast((cast(1 as double) / cast(0 as double)) as string)) AS DOUBLE) ELSE CAST(CAST(udf(cast(0 as string)) AS INT) AS DOUBLE) END:double> -- !query 20 output -0 -0 -0 -0 +0.0 +0.0 +0.0 +0.0 -- !query 21 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out index a60cbf33b9b24..2540f0260f09c 100755 --- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -242,31 +242,36 @@ struct +struct -- !query 23 output -0 0 -0 0 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 24 SELECT udf(a/2), udf(a/2) FROM test_missing_target GROUP BY udf(a/2) ORDER BY udf(a/2) -- !query 24 schema -struct +struct -- !query 24 output -0 0 -1 1 -2 2 -3 3 -4 4 +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 -- !query 25 @@ -334,7 +339,8 @@ SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2 struct -- !query 30 output 1 -5 +2 +3 4 @@ -373,8 +379,10 @@ SELECT udf(count(b)) FROM test_missing_target -- !query 33 schema struct -- !query 33 output -7 +4 3 +2 +1 -- !query 34 @@ -394,11 +402,12 @@ test_missing_target y WHERE udf(x.a) = udf(y.a) GROUP BY udf(x.b/2) ORDER BY udf(x.b/2) -- !query 35 schema -struct +struct -- !query 35 output -0 1 -1 5 -2 4 +0.5 1 +1.0 2 +1.5 3 +2.0 4 -- !query 36 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 53423a077f39f..9169b3819f0a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -333,6 +333,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession { // vol used by boolean.sql and case.sql. localSparkSession.udf.register("vol", (s: String) => s) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + case _: AnsiTest => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) case _ => } From c50130a8042f7f89b6bd606251488003bfda995e Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Wed, 4 Dec 2019 21:18:55 +0800 Subject: [PATCH 11/16] fix ThriftServerQueryTestSuite --- .../sql/hive/thriftserver/ThriftServerQueryTestSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 7b208a3b0233f..7d2963f3c21f0 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -117,7 +117,7 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite { } testCase match { - case _: PgSQLTest => + case _: PgSQLTest | _: AnsiTest => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") case _ => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") From cf2bc65ec47cfc7db0d8077c41816b2825282461 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 16 Oct 2019 06:26:01 -0700 Subject: [PATCH 12/16] [SPARK-29364][SQL] Return an interval from date subtract according to SQL standard Proposed new expression `SubtractDates` which is used in `date1` - `date2`. It has the `INTERVAL` type, and returns the interval from `date1` (inclusive) and `date2` (exclusive). For example: ```sql > select date'tomorrow' - date'yesterday'; interval 2 days ``` Closes #26034 - To conform the SQL standard which states the result type of `date operand 1` - `date operand 2` must be the interval type. See [4.5.3 Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt). - Improve Spark SQL UX and allow mixing date and timestamp in subtractions. For example: `select timestamp'now' + (date'2019-10-01' - date'2019-09-15')` Before the query below returns number of days: ```sql spark-sql> select date'2019-10-05' - date'2018-09-01'; 399 ``` After it returns an interval: ```sql spark-sql> select date'2019-10-05' - date'2018-09-01'; interval 1 years 1 months 4 days ``` - by new tests in `DateExpressionsSuite` and `TypeCoercionSuite`. - by existing tests in `date.sql` Closes #26112 from MaxGekk/date-subtract. Authored-by: Maxim Gekk Signed-off-by: Yuming Wang --- .../expressions/datetimeExpressions.scala | 22 +++++++++++++ .../sql/catalyst/util/DateTimeUtils.scala | 18 ++++++++++- .../expressions/DateExpressionsSuite.scala | 31 ++++++++++++++++--- 3 files changed, 65 insertions(+), 6 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 a36dc48017acd..b3aeda852a4dd 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 @@ -2145,3 +2145,25 @@ case class SubtractTimestamps(endTimestamp: Expression, startTimestamp: Expressi s"new org.apache.spark.unsafe.types.CalendarInterval(0, 0, $end - $start)") } } + +/** + * Returns the interval from the `left` date (inclusive) to the `right` date (exclusive). + */ +case class SubtractDates(left: Expression, right: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, DateType) + override def dataType: DataType = CalendarIntervalType + + override def nullSafeEval(leftDays: Any, rightDays: Any): Any = { + DateTimeUtils.subtractDates(leftDays.asInstanceOf[Int], rightDays.asInstanceOf[Int]) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (leftDays, rightDays) => { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + s"$dtu.subtractDates($leftDays, $rightDays)" + }) + } +} + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index e5e25e251f128..8ebacd1c71def 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.types.Decimal -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} /** * Helper functions for converting between internal and external date and time representations. @@ -939,4 +939,20 @@ object DateTimeUtils { None } } + + /** + * Subtracts two dates. + * @param endDate - the end date, exclusive + * @param startDate - the start date, inclusive + * @return an interval between two dates. The interval can be negative + * if the end date is before the start date. + */ + def subtractDates(endDate: SQLDate, startDate: SQLDate): CalendarInterval = { + val period = Period.between( + LocalDate.ofEpochDay(startDate), + LocalDate.ofEpochDay(endDate)) + val months = period.getMonths + 12 * period.getYears + val microseconds = period.getDays * MICROS_PER_DAY + new CalendarInterval(months, microseconds) + } } 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 64d7ea5e8b791..2b5404099dc03 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Instant, LocalDateTime, ZoneId, ZoneOffset} +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit._ @@ -1103,19 +1103,40 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("timestamps difference") { val end = Instant.parse("2019-10-04T11:04:01.123456Z") - checkEvaluation(TimestampDiff(Literal(end), Literal(end)), + checkEvaluation(SubtractTimestamps(Literal(end), Literal(end)), new CalendarInterval(0, 0, 0)) - checkEvaluation(TimestampDiff(Literal(end), Literal(Instant.EPOCH)), + checkEvaluation(SubtractTimestamps(Literal(end), Literal(Instant.EPOCH)), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "436163 hours 4 minutes 1 seconds 123 milliseconds 456 microseconds"))) - checkEvaluation(TimestampDiff(Literal(Instant.EPOCH), Literal(end)), + checkEvaluation(SubtractTimestamps(Literal(Instant.EPOCH), Literal(end)), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "-436163 hours -4 minutes -1 seconds -123 milliseconds -456 microseconds"))) checkEvaluation( - TimestampDiff( + SubtractTimestamps( Literal(Instant.parse("9999-12-31T23:59:59.999999Z")), Literal(Instant.parse("0001-01-01T00:00:00Z"))), IntervalUtils.stringToInterval(UTF8String.fromString("interval " + "87649415 hours 59 minutes 59 seconds 999 milliseconds 999 microseconds"))) } + + test("subtract dates") { + val end = LocalDate.of(2019, 10, 5) + checkEvaluation(SubtractDates(Literal(end), Literal(end)), + new CalendarInterval(0, 0)) + checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), + IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 days"))) + checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), + IntervalUtils.stringToInterval(UTF8String.fromString("interval -1 days"))) + val epochDate = Literal(LocalDate.ofEpochDay(0)) + checkEvaluation(SubtractDates(Literal(end), epochDate), + IntervalUtils.stringToInterval(UTF8String.fromString("interval 49 years 9 months 4 days"))) + checkEvaluation(SubtractDates(epochDate, Literal(end)), + IntervalUtils.stringToInterval(UTF8String.fromString( + "interval -49 years -9 months -4 days"))) + checkEvaluation( + SubtractDates( + Literal(LocalDate.of(10000, 1, 1)), + Literal(LocalDate.of(1, 1, 1))), + IntervalUtils.stringToInterval(UTF8String.fromString("interval 9999 years"))) + } } From 1f90047e7ad2271f9a1de2bcb5a2a93e8e574676 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 21 Nov 2019 00:56:48 +0800 Subject: [PATCH 13/16] [SPARK-29951][SQL] Make the behavior of Postgre dialect independent of ansi mode config Fix the inconsistent behavior of build-in function SQL LEFT/RIGHT. As the comment in https://github.com/apache/spark/pull/26497#discussion_r345708065, Postgre dialect should not be affected by the ANSI mode config. During reran the existing tests, only the LEFT/RIGHT build-in SQL function broke the assumption. We fix this by following https://www.postgresql.org/docs/12/sql-keywords-appendix.html: `LEFT/RIGHT reserved (can be function or type)` Yes, the Postgre dialect will not be affected by the ANSI mode config. Existing UT. Closes #26584 from xuanyuanking/SPARK-29951. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan --- docs/sql-keywords.md | 7 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 26 +-- .../sql/catalyst/parser/AstBuilder.scala | 16 +- .../sql/catalyst/parser/ParseDriver.scala | 1 + .../sql-tests/inputs/postgreSQL/text.sql | 4 - .../sql-tests/results/postgreSQL/text.sql.out | 160 ++++++++---------- 6 files changed, 107 insertions(+), 107 deletions(-) mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 85dffd6ea6d66..88a607211aad9 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,12 +19,13 @@ license: | limitations under the License. --- -When `spark.sql.ansi.enabled` is true, Spark SQL has two kinds of keywords: +When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. +In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. -When `spark.sql.ansi.enabled` is false, Spark SQL has two kinds of keywords: -* Non-reserved keywords: Same definition as the one when `spark.sql.ansi.enabled=true`. +When the ANSI mode is disabled, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when the ANSI mode enabled. * Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. By default `spark.sql.ansi.enabled` is false. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 15be6da14f5ec..f894836d64233 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -58,9 +58,9 @@ grammar SqlBase; } /** - * When true, ANSI SQL parsing mode is enabled. + * When true, the behavior of keywords follows ANSI SQL standard. */ - public boolean ansi = false; + public boolean SQL_standard_keyword_behavior = false; } singleStatement @@ -766,7 +766,7 @@ primaryExpression | qualifiedName '.' ASTERISK #star | '(' namedExpression (',' namedExpression)+ ')' #rowConstructor | '(' query ')' #subqueryExpression - | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' + | functionName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall | identifier '->' expression #lambda | '(' identifier (',' identifier)+ ')' '->' expression #lambda @@ -810,7 +810,7 @@ booleanValue interval : INTERVAL (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval)? - | {ansi}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) + | {SQL_standard_keyword_behavior}? (errorCapturingMultiUnitsInterval | errorCapturingUnitToUnitInterval) ; errorCapturingMultiUnitsInterval @@ -930,6 +930,12 @@ qualifiedNameList : qualifiedName (',' qualifiedName)* ; +functionName + : qualifiedName + | LEFT + | RIGHT + ; + qualifiedName : identifier ('.' identifier)* ; @@ -949,14 +955,14 @@ errorCapturingIdentifierExtra identifier : strictIdentifier - | {!ansi}? strictNonReserved + | {!SQL_standard_keyword_behavior}? strictNonReserved ; strictIdentifier : IDENTIFIER #unquotedIdentifier | quotedIdentifier #quotedIdentifierAlternative - | {ansi}? ansiNonReserved #unquotedIdentifier - | {!ansi}? nonReserved #unquotedIdentifier + | {SQL_standard_keyword_behavior}? ansiNonReserved #unquotedIdentifier + | {!SQL_standard_keyword_behavior}? nonReserved #unquotedIdentifier ; quotedIdentifier @@ -975,7 +981,7 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -// When `spark.sql.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=true`, there are 2 kinds of keywords in Spark SQL. // - Reserved keywords: // Keywords that are reserved and can't be used as identifiers for table, view, column, // function, alias, etc. @@ -1175,9 +1181,9 @@ ansiNonReserved | YEARS ; -// When `spark.sql.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// When `SQL_standard_keyword_behavior=false`, there are 2 kinds of keywords in Spark SQL. // - Non-reserved keywords: -// Same definition as the one when `spark.sql.ansi.enabled=true`. +// Same definition as the one when `SQL_standard_keyword_behavior=true`. // - Strict-non-reserved keywords: // A strict version of non-reserved keywords, which can not be used as table alias. // You can find the full keywords list by searching "Start of the keywords list" in this file. 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 3b5b44d926d0f..858870a161417 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 @@ -1589,7 +1589,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { // Create the function call. - val name = ctx.qualifiedName.getText + val name = ctx.functionName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) val arguments = ctx.argument.asScala.map(expression) match { case Seq(UnresolvedStar(None)) @@ -1599,7 +1599,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case expressions => expressions } - val function = UnresolvedFunction(visitFunctionName(ctx.qualifiedName), arguments, isDistinct) + val function = UnresolvedFunction( + getFunctionIdentifier(ctx.functionName), arguments, isDistinct) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { @@ -1639,6 +1640,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Get a function identifier consist by database (optional) and name. + */ + protected def getFunctionIdentifier(ctx: FunctionNameContext): FunctionIdentifier = { + if (ctx.qualifiedName != null) { + visitFunctionName(ctx.qualifiedName) + } else { + FunctionIdentifier(ctx.getText, None) + } + } + /** * Create an [[LambdaFunction]]. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index c96f2da9a5289..5488cfb172729 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql index 7abf903bc6bee..05953123da86f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/text.sql @@ -44,11 +44,7 @@ select concat_ws(',',10,20,null,30); select concat_ws('',10,20,null,30); select concat_ws(NULL,10,20,null,30) is null; select reverse('abcde'); --- [SPARK-28036] Built-in udf left/right has inconsistent behavior --- [SPARK-28479][SPARK-28989] Parser error when enabling ANSI mode -set spark.sql.ansi.enabled=false; select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; -set spark.sql.ansi.enabled=true; -- [SPARK-28037] Add built-in String Functions: quote_literal -- select quote_literal(''); -- select quote_literal('abc'''); diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out old mode 100644 new mode 100755 index 2e1d639974ec6..311b0eb5a5844 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/text.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 44 +-- Number of queries: 42 -- !query 0 @@ -151,18 +151,10 @@ edcba -- !query 18 -set spark.sql.ansi.enabled=false --- !query 18 schema -struct --- !query 18 output -spark.sql.ansi.enabled false - - --- !query 19 select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i --- !query 19 schema +-- !query 18 schema struct --- !query 19 output +-- !query 18 output -5 -4 -3 @@ -176,200 +168,192 @@ struct 5 ahoj ahoj --- !query 20 -set spark.sql.ansi.enabled=true --- !query 20 schema -struct --- !query 20 output -spark.sql.ansi.enabled true - - --- !query 21 +-- !query 19 /* * format */ select format_string(NULL) --- !query 21 schema +-- !query 19 schema struct --- !query 21 output +-- !query 19 output NULL --- !query 22 +-- !query 20 select format_string('Hello') --- !query 22 schema +-- !query 20 schema struct --- !query 22 output +-- !query 20 output Hello --- !query 23 +-- !query 21 select format_string('Hello %s', 'World') --- !query 23 schema +-- !query 21 schema struct --- !query 23 output +-- !query 21 output Hello World --- !query 24 +-- !query 22 select format_string('Hello %%') --- !query 24 schema +-- !query 22 schema struct --- !query 24 output +-- !query 22 output Hello % --- !query 25 +-- !query 23 select format_string('Hello %%%%') --- !query 25 schema +-- !query 23 schema struct --- !query 25 output +-- !query 23 output Hello %% --- !query 26 +-- !query 24 select format_string('Hello %s %s', 'World') --- !query 26 schema +-- !query 24 schema struct<> --- !query 26 output +-- !query 24 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 27 +-- !query 25 select format_string('Hello %s') --- !query 27 schema +-- !query 25 schema struct<> --- !query 27 output +-- !query 25 output java.util.MissingFormatArgumentException Format specifier '%s' --- !query 28 +-- !query 26 select format_string('Hello %x', 20) --- !query 28 schema +-- !query 26 schema struct --- !query 28 output +-- !query 26 output Hello 14 --- !query 29 +-- !query 27 select format_string('%1$s %3$s', 1, 2, 3) --- !query 29 schema +-- !query 27 schema struct --- !query 29 output +-- !query 27 output 1 3 --- !query 30 +-- !query 28 select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 30 schema +-- !query 28 schema struct --- !query 30 output +-- !query 28 output 1 12 --- !query 31 +-- !query 29 select format_string('%1$s %4$s', 1, 2, 3) --- !query 31 schema +-- !query 29 schema struct<> --- !query 31 output +-- !query 29 output java.util.MissingFormatArgumentException Format specifier '%4$s' --- !query 32 +-- !query 30 select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) --- !query 32 schema +-- !query 30 schema struct<> --- !query 32 output +-- !query 30 output java.util.MissingFormatArgumentException Format specifier '%13$s' --- !query 33 +-- !query 31 select format_string('%0$s', 'Hello') --- !query 33 schema +-- !query 31 schema struct --- !query 33 output +-- !query 31 output Hello --- !query 34 +-- !query 32 select format_string('Hello %s %1$s %s', 'World', 'Hello again') --- !query 34 schema +-- !query 32 schema struct --- !query 34 output +-- !query 32 output Hello World World Hello again --- !query 35 +-- !query 33 select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') --- !query 35 schema +-- !query 33 schema struct --- !query 35 output +-- !query 33 output Hello World Hello again, Hello again Hello again --- !query 36 +-- !query 34 select format_string('>>%10s<<', 'Hello') --- !query 36 schema +-- !query 34 schema struct>%10s<<, Hello):string> --- !query 36 output +-- !query 34 output >> Hello<< --- !query 37 +-- !query 35 select format_string('>>%10s<<', NULL) --- !query 37 schema +-- !query 35 schema struct>%10s<<, NULL):string> --- !query 37 output +-- !query 35 output >> null<< --- !query 38 +-- !query 36 select format_string('>>%10s<<', '') --- !query 38 schema +-- !query 36 schema struct>%10s<<, ):string> --- !query 38 output +-- !query 36 output >> << --- !query 39 +-- !query 37 select format_string('>>%-10s<<', '') --- !query 39 schema +-- !query 37 schema struct>%-10s<<, ):string> --- !query 39 output +-- !query 37 output >> << --- !query 40 +-- !query 38 select format_string('>>%-10s<<', 'Hello') --- !query 40 schema +-- !query 38 schema struct>%-10s<<, Hello):string> --- !query 40 output +-- !query 38 output >>Hello << --- !query 41 +-- !query 39 select format_string('>>%-10s<<', NULL) --- !query 41 schema +-- !query 39 schema struct>%-10s<<, NULL):string> --- !query 41 output +-- !query 39 output >>null << --- !query 42 +-- !query 40 select format_string('>>%1$10s<<', 'Hello') --- !query 42 schema +-- !query 40 schema struct>%1$10s<<, Hello):string> --- !query 42 output +-- !query 40 output >> Hello<< --- !query 43 +-- !query 41 DROP TABLE TEXT_TBL --- !query 43 schema +-- !query 41 schema struct<> --- !query 43 output +-- !query 41 output From cc1427457f4e3092ef514f8768adceca49c9668c Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 9 Dec 2019 19:22:14 +0800 Subject: [PATCH 14/16] fix --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 +++++- .../org/apache/spark/sql/catalyst/parser/ParseDriver.scala | 1 - .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 4 ++-- .../sql/catalyst/expressions/DateExpressionsSuite.scala | 2 +- .../test/resources/sql-tests/results/ansi/interval.sql.out | 4 ++-- .../src/test/resources/sql-tests/results/interval.sql.out | 4 ++-- .../resources/sql-tests/results/postgreSQL/boolean.sql.out | 4 ++-- 7 files changed, 14 insertions(+), 11 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 e292514667500..4916945ed0c8f 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 @@ -286,7 +286,11 @@ class Analyzer( case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) case (TimestampType, _) => SubtractTimestamps(l, r) case (_, TimestampType) => SubtractTimestamps(l, r) - case (_, DateType) => SubtractDates(l, r) + case (_, DateType) => if (conf.ansiEnabled) { + DateDiff(l, r) + } else { + SubtractDates(l, r) + } case (DateType, _) => DateSub(l, r) case _ => s } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 5488cfb172729..c96f2da9a5289 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.Dialect import org.apache.spark.sql.types.{DataType, StructType} /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 8ebacd1c71def..6800abb2ae109 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -952,7 +952,7 @@ object DateTimeUtils { LocalDate.ofEpochDay(startDate), LocalDate.ofEpochDay(endDate)) val months = period.getMonths + 12 * period.getYears - val microseconds = period.getDays * MICROS_PER_DAY - new CalendarInterval(months, microseconds) + val days = period.getDays + new CalendarInterval(months, days, 0) } } 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 2b5404099dc03..55545395da07d 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 @@ -1122,7 +1122,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("subtract dates") { val end = LocalDate.of(2019, 10, 5) checkEvaluation(SubtractDates(Literal(end), Literal(end)), - new CalendarInterval(0, 0)) + new CalendarInterval(0, 0, 0)) checkEvaluation(SubtractDates(Literal(end.plusDays(1)), Literal(end)), IntervalUtils.stringToInterval(UTF8String.fromString("interval 1 days"))) checkEvaluation(SubtractDates(Literal(end.minusDays(1)), Literal(end)), 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 7d94711ded343..2709fb42488cc 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 @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out index 04e1d8dd95848..601521ba43227 100644 --- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out @@ -183,7 +183,7 @@ struct -- !query 22 select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') -- !query 22 schema -struct +struct -- !query 22 output 30 hours 33 minutes 36.003006 seconds @@ -199,7 +199,7 @@ struct +struct -- !query 24 output 16 hours diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out index c7903c8a34ef4..39f239d7dbf2e 100644 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/boolean.sql.out @@ -39,7 +39,7 @@ SELECT boolean(' f ') AS `false` -- !query 4 schema struct -- !query 4 output -NULL +false -- !query 5 @@ -296,7 +296,7 @@ SELECT boolean(string(' true ')) AS true, -- !query 36 schema struct -- !query 36 output -NULL NULL +true false -- !query 37 From 6d978c673df5d74751a6fac822157788e025312d Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Mon, 9 Dec 2019 22:12:10 +0800 Subject: [PATCH 15/16] fix unnecessary change --- .../org/apache/spark/sql/catalyst/util/StringUtils.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index 76fe682ef7ccf..b42ae4e45366e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -38,7 +38,7 @@ object StringUtils extends Logging { * throw an [[AnalysisException]]. * * @param pattern the SQL pattern to convert - * @param escapeStr the escape string contains one character. + * @param escapeChar the escape string contains one character. * @return the equivalent Java regular expression of the pattern */ def escapeLikeRegex(pattern: String, escapeChar: Char): String = { @@ -66,8 +66,11 @@ object StringUtils extends Logging { "(?s)" + out.result() // (?s) enables dotall mode, causing "." to match new lines } - private[this] val trueStrings = Set("t", "true", "y", "yes", "1").map(UTF8String.fromString) - private[this] val falseStrings = Set("f", "false", "n", "no", "0").map(UTF8String.fromString) + private[this] val trueStrings = + Set("t", "true", "y", "yes", "1").map(UTF8String.fromString) + + private[this] val falseStrings = + Set("f", "false", "n", "no", "0").map(UTF8String.fromString) // scalastyle:off caselocale def isTrueString(s: UTF8String): Boolean = trueStrings.contains(s.trimAll().toLowerCase) From 17a3c778c1c584d62e81acedc4742b2e50a83027 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 10 Dec 2019 18:24:34 +0800 Subject: [PATCH 16/16] address comment from Wenchen --- docs/sql-keywords.md | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../expressions/DateExpressionsSuite.scala | 3 +- .../sql-tests/inputs/postgreSQL/boolean.sql | 4 - .../sql-tests/inputs/postgreSQL/int2.sql | 6 +- .../sql-tests/inputs/postgreSQL/int4.sql | 1 - .../sql-tests/inputs/postgreSQL/int8.sql | 1 - .../sql-tests/results/postgreSQL/date.sql.out | 88 +++++++++---------- .../sql-tests/results/postgreSQL/int2.sql.out | 28 +++--- 9 files changed, 63 insertions(+), 76 deletions(-) mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out mode change 100644 => 100755 sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md index 88a607211aad9..4f50ba6d440cb 100644 --- a/docs/sql-keywords.md +++ b/docs/sql-keywords.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -When `spark.sql.dialect.spark.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. +When `spark.sql.ansi.enabled` is true, Spark SQL will use the ANSI mode parser. In this mode, Spark SQL has two kinds of keywords: * Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. * Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. 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 4916945ed0c8f..e292514667500 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 @@ -286,11 +286,7 @@ class Analyzer( case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType) case (TimestampType, _) => SubtractTimestamps(l, r) case (_, TimestampType) => SubtractTimestamps(l, r) - case (_, DateType) => if (conf.ansiEnabled) { - DateDiff(l, r) - } else { - SubtractDates(l, r) - } + case (_, DateType) => SubtractDates(l, r) case (DateType, _) => DateSub(l, r) case _ => s } 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 55545395da07d..7d015aa478df0 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 @@ -1131,8 +1131,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(SubtractDates(Literal(end), epochDate), IntervalUtils.stringToInterval(UTF8String.fromString("interval 49 years 9 months 4 days"))) checkEvaluation(SubtractDates(epochDate, Literal(end)), - IntervalUtils.stringToInterval(UTF8String.fromString( - "interval -49 years -9 months -4 days"))) + IntervalUtils.stringToInterval(UTF8String.fromString("interval -49 years -9 months -4 days"))) checkEvaluation( SubtractDates( Literal(LocalDate.of(10000, 1, 1)), diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql index 4427d76f48d80..3a949c834deb5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/boolean.sql @@ -22,7 +22,6 @@ SELECT false AS `false`; SELECT boolean('t') AS true; --- [SPARK-27931] Trim the string when cast string type to boolean type SELECT boolean(' f ') AS `false`; SELECT boolean('true') AS true; @@ -49,12 +48,10 @@ SELECT boolean('no') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL SELECT boolean('nay') AS error; --- [SPARK-27931] Accept 'on' and 'off' as input for boolean data type SELECT boolean('on') AS true; SELECT boolean('off') AS `false`; --- [SPARK-27931] Accept unique prefixes thereof SELECT boolean('of') AS `false`; -- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL @@ -101,7 +98,6 @@ SELECT boolean('f') <= boolean('t') AS true; -- explicit casts to/from text SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; --- [SPARK-27931] Trim the string when cast to boolean type SELECT boolean(string(' true ')) AS true, boolean(string(' FALSE')) AS `false`; SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql index 835e02c36436a..07f5976ca6d2f 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int2.sql @@ -92,11 +92,9 @@ WHERE f1 > -32767; SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; --- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. -SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i; +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; --- PostgreSQL `/` is the same with Spark `div` since SPARK-2659. -SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i; +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; -- corner cases SELECT string(shiftleft(smallint(-1), 15)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql index 8df3a0e698cbe..3a409eea34837 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int4.sql @@ -138,7 +138,6 @@ SELECT int('1000') < int('999') AS `false`; SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; --- [SPARK-2659] HiveQL: Division operator should always perform fractional division SELECT 2 + 2 / 2 AS three; SELECT (2 + 2) / 2 AS two; diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql index ed46ae30ad283..5fea758e73084 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql @@ -87,7 +87,6 @@ SELECT 37 - q1 AS minus4 FROM INT8_TBL; SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; --- [SPARK-2659] HiveQL: Division operator should always perform fractional division -- int8 op int4 SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; -- int4 op int8 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out old mode 100644 new mode 100755 index 5371e07423c28..700476e91eaac --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/date.sql.out @@ -482,93 +482,93 @@ SELECT date '5874898-01-01' -- !query 46 SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL -- !query 46 schema -struct +struct -- !query 46 output --1035 --1036 --1037 --1400 --1401 --1402 --1403 --15542 --15607 -13977 -14343 -14710 -91 -92 -93 +-2 years -10 months +-2 years -10 months -1 days +-2 years -9 months -30 days +-3 years -10 months +-3 years -10 months -1 days +-3 years -10 months -2 days +-3 years -9 months -30 days +-42 years -6 months -18 days +-42 years -8 months -22 days +3 months +3 months 1 days +3 months 2 days +38 years 3 months 7 days +39 years 3 months 8 days +40 years 3 months 9 days -- !query 47 SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL -- !query 47 schema -struct +struct -- !query 47 output --4585 --4650 -11048 -11049 -11050 -24934 -25300 -25667 -9554 -9555 -9556 -9557 -9920 -9921 -9922 +-12 years -6 months -18 days +-12 years -8 months -22 days +26 years 1 months 27 days +26 years 1 months 28 days +26 years 2 months +26 years 2 months 1 days +27 years 1 months 27 days +27 years 2 months +27 years 2 months 1 days +30 years 3 months +30 years 3 months 1 days +30 years 3 months 2 days +68 years 3 months 7 days +69 years 3 months 8 days +70 years 3 months 9 days -- !query 48 SELECT date 'yesterday' - date 'today' AS `One day` -- !query 48 schema -struct +struct -- !query 48 output --1 +-1 days -- !query 49 SELECT date 'today' - date 'tomorrow' AS `One day` -- !query 49 schema -struct +struct -- !query 49 output --1 +-1 days -- !query 50 SELECT date 'yesterday' - date 'tomorrow' AS `Two days` -- !query 50 schema -struct +struct -- !query 50 output --2 +-2 days -- !query 51 SELECT date 'tomorrow' - date 'today' AS `One day` -- !query 51 schema -struct +struct -- !query 51 output -1 +1 days -- !query 52 SELECT date 'today' - date 'yesterday' AS `One day` -- !query 52 schema -struct +struct -- !query 52 output -1 +1 days -- !query 53 SELECT date 'tomorrow' - date 'yesterday' AS `Two days` -- !query 53 schema -struct +struct -- !query 53 output -2 +2 days -- !query 54 diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out old mode 100644 new mode 100755 index d7a3fdf481418..d6e9f7ec79d68 --- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int2.sql.out @@ -266,27 +266,27 @@ struct -- !query 27 -SELECT '' AS five, i.f1, i.f1 div smallint('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i -- !query 27 schema -struct +struct -- !query 27 output - -1234 -617 - -32767 -16383 - 0 0 - 1234 617 - 32767 16383 + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 -- !query 28 -SELECT '' AS five, i.f1, i.f1 div int('2') AS x FROM INT2_TBL i +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i -- !query 28 schema -struct +struct -- !query 28 output - -1234 -617 - -32767 -16383 - 0 0 - 1234 617 - 32767 16383 + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 -- !query 29