From b6265286eca9b8ad22cf01da9577d36b0ac0ada5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 18 Nov 2024 10:05:45 +0800 Subject: [PATCH 01/58] [SPARK-50313][SQL][TESTS] Enable ANSI in SQL *SQLQueryTestSuite by default ### What changes were proposed in this pull request? This PR enables ANSI in SQL *SQLQueryTestSuite by default for regular tests and switches the original ansi/ ones to nonansi, following SPARK-44444, which turn on ansi for prod, ### Why are the changes needed? test improvements ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? new golden files ### Was this patch authored or co-authored using generative AI tooling? no Closes #48842 from yaooqinn/SPARK-50313. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../sql-tests/analyzer-results/array.sql.out | 28 +- .../sql-tests/analyzer-results/cast.sql.out | 189 +- .../analyzer-results/collations.sql.out | 10 +- .../analyzer-results/comparator.sql.out | 10 +- .../analyzer-results/csv-functions.sql.out | 2 +- .../sql-tests/analyzer-results/date.sql.out | 147 +- .../analyzer-results/datetime-legacy.sql.out | 333 +-- .../datetime-parsing-invalid.sql.out | 52 +- .../datetime-parsing-legacy.sql.out | 52 +- .../analyzer-results/datetime-parsing.sql.out | 52 +- .../analyzer-results/datetime-special.sql.out | 4 +- .../decimalArithmeticOperations.sql.out | 9 +- .../analyzer-results/extract.sql.out | 198 +- .../group-by-all-mosha.sql.out | 2 +- .../analyzer-results/group-by-filter.sql.out | 14 +- .../analyzer-results/group-by.sql.out | 21 +- .../higher-order-functions.sql.out | 6 +- .../analyzer-results/interval.sql.out | 18 +- .../analyzer-results/json-functions.sql.out | 2 +- .../sql-tests/analyzer-results/map.sql.out | 2 +- .../sql-tests/analyzer-results/math.sql.out | 12 +- .../{ansi => nonansi}/array.sql.out | 28 +- .../{ansi => nonansi}/cast.sql.out | 189 +- .../conditional-functions.sql.out | 2 +- .../{ansi => nonansi}/date.sql.out | 147 +- .../datetime-parsing-invalid.sql.out | 52 +- .../datetime-special.sql.out | 4 +- .../decimalArithmeticOperations.sql.out | 0 ...double-quoted-identifiers-disabled.sql.out | 0 .../double-quoted-identifiers-enabled.sql.out | 261 +-- .../higher-order-functions.sql.out | 6 +- .../{ansi => nonansi}/interval.sql.out | 18 +- .../{ansi => nonansi}/keywords.sql.out | 0 .../{ansi => nonansi}/literals.sql.out | 0 .../{ansi => nonansi}/map.sql.out | 2 +- .../{ansi => nonansi}/math.sql.out | 12 +- .../parse-schema-string.sql.out | 0 .../string-functions.sql.out | 0 .../{ansi => nonansi}/timestamp.sql.out | 186 +- .../{ansi => nonansi}/try_aggregates.sql.out | 0 .../{ansi => nonansi}/try_arithmetic.sql.out | 6 +- .../try_datetime_functions.sql.out | 0 .../{ansi => nonansi}/try_element_at.sql.out | 0 .../analyzer-results/pipe-operators.sql.out | 9 +- .../predicate-functions.sql.out | 66 +- .../subexp-elimination.sql.out | 2 +- .../scalar-subquery-predicate.sql.out | 2 +- .../analyzer-results/timestamp-ltz.sql.out | 12 +- .../analyzer-results/timestamp-ntz.sql.out | 10 +- .../analyzer-results/timestamp.sql.out | 186 +- .../analyzer-results/transform.sql.out | 4 +- .../analyzer-results/try_arithmetic.sql.out | 6 +- .../native/binaryComparison.sql.out | 840 ++++--- .../native/booleanEquality.sql.out | 1496 ++++++++---- .../native/caseWhenCoercion.sql.out | 144 +- .../typeCoercion/native/concat.sql.out | 2 +- .../native/decimalPrecision.sql.out | 332 ++- .../typeCoercion/native/division.sql.out | 89 +- .../typeCoercion/native/elt.sql.out | 8 +- .../typeCoercion/native/ifCoercion.sql.out | 144 +- .../native/implicitTypeCasts.sql.out | 8 +- .../typeCoercion/native/inConversion.sql.out | 288 +-- .../typeCoercion/native/mapZipWith.sql.out | 100 +- .../typeCoercion/native/mapconcat.sql.out | 24 +- .../native/promoteStrings.sql.out | 711 +++--- .../native/stringCastAndExpressions.sql.out | 16 +- .../native/widenSetOperationTypes.sql.out | 618 +++-- .../native/windowFrameCoercion.sql.out | 9 +- .../udf/udf-cross-join.sql.out | 2 +- .../analyzer-results/udf/udf-group-by.sql.out | 21 +- .../analyzer-results/udf/udf-union.sql.out | 29 +- .../sql-tests/analyzer-results/union.sql.out | 16 +- .../analyzer-results/url-functions.sql.out | 16 +- .../sql-tests/analyzer-results/window.sql.out | 12 +- .../analyzer-results/xml-functions.sql.out | 2 +- .../inputs/decimalArithmeticOperations.sql | 2 +- .../inputs/{ansi => nonansi}/array.sql | 0 .../inputs/{ansi => nonansi}/cast.sql | 0 .../conditional-functions.sql | 0 .../inputs/{ansi => nonansi}/date.sql | 0 .../datetime-parsing-invalid.sql | 0 .../{ansi => nonansi}/datetime-special.sql | 0 .../decimalArithmeticOperations.sql | 0 .../double-quoted-identifiers-disabled.sql | 0 .../double-quoted-identifiers-enabled.sql | 0 .../higher-order-functions.sql | 0 .../inputs/{ansi => nonansi}/interval.sql | 0 .../inputs/{ansi => nonansi}/keywords.sql | 0 .../inputs/{ansi => nonansi}/literals.sql | 0 .../inputs/{ansi => nonansi}/map.sql | 0 .../inputs/{ansi => nonansi}/math.sql | 0 .../{ansi => nonansi}/parse-schema-string.sql | 0 .../{ansi => nonansi}/string-functions.sql | 0 .../inputs/{ansi => nonansi}/timestamp.sql | 0 .../{ansi => nonansi}/try_aggregates.sql | 0 .../{ansi => nonansi}/try_arithmetic.sql | 0 .../try_datetime_functions.sql | 0 .../{ansi => nonansi}/try_element_at.sql | 0 .../sql-tests/inputs/pipe-operators.sql | 3 +- .../sql-tests/inputs/udf/udf-union.sql | 2 +- .../sql-tests/results/ansi/cast.sql.out | 1999 ----------------- .../ansi/datetime-parsing-invalid.sql.out | 465 ---- .../ansi/decimalArithmeticOperations.sql.out | 365 --- .../sql-tests/results/ansi/math.sql.out | 903 -------- .../ansi/try_aggregates.sql.out.java21 | 415 ---- .../resources/sql-tests/results/array.sql.out | 151 +- .../resources/sql-tests/results/cast.sql.out | 1055 ++++++++- .../resources/sql-tests/results/date.sql.out | 201 +- .../sql-tests/results/datetime-legacy.sql.out | 624 +++-- .../results/datetime-parsing-invalid.sql.out | 328 ++- .../decimalArithmeticOperations.sql.out | 437 +++- .../results/group-by-all-mosha.sql.out | 22 +- .../sql-tests/results/group-by.sql.out | 22 +- .../sql-tests/results/interval.sql.out | 221 +- .../resources/sql-tests/results/math.sql.out | 404 +++- .../results/{ansi => nonansi}/array.sql.out | 151 +- .../sql-tests/results/nonansi/cast.sql.out | 1156 ++++++++++ .../conditional-functions.sql.out | 40 +- .../results/{ansi => nonansi}/date.sql.out | 201 +- .../nonansi/datetime-parsing-invalid.sql.out | 241 ++ .../datetime-special.sql.out | 0 .../decimalArithmeticOperations.sql.out | 195 ++ ...double-quoted-identifiers-disabled.sql.out | 0 .../double-quoted-identifiers-enabled.sql.out | 238 +- .../higher-order-functions.sql.out | 0 .../{ansi => nonansi}/interval.sql.out | 221 +- .../{ansi => nonansi}/keywords.sql.out | 229 +- .../{ansi => nonansi}/literals.sql.out | 0 .../results/{ansi => nonansi}/map.sql.out | 0 .../sql-tests/results/nonansi/math.sql.out | 583 +++++ .../parse-schema-string.sql.out | 0 .../string-functions.sql.out | 80 +- .../{ansi => nonansi}/timestamp.sql.out | 219 +- .../{ansi => nonansi}/try_aggregates.sql.out | 148 +- .../{ansi => nonansi}/try_arithmetic.sql.out | 244 +- .../try_datetime_functions.sql.out | 0 .../{ansi => nonansi}/try_element_at.sql.out | 0 .../sql-tests/results/operators.sql.out | 112 +- .../sql-tests/results/pipe-operators.sql.out | 23 +- .../results/predicate-functions.sql.out | 184 +- .../results/string-functions.sql.out | 80 +- .../results/subexp-elimination.sql.out | 10 +- .../sql-tests/results/timestamp-ltz.sql.out | 11 +- .../sql-tests/results/timestamp-ntz.sql.out | 11 +- .../sql-tests/results/timestamp.sql.out | 219 +- .../sql-tests/results/transform.sql.out | 8 +- .../sql-tests/results/try_aggregates.sql.out | 148 +- .../sql-tests/results/try_arithmetic.sql.out | 244 +- .../native/binaryComparison.sql.out | 528 ++++- .../native/booleanEquality.sql.out | 1134 +++++++++- .../native/caseWhenCoercion.sql.out | 162 +- .../typeCoercion/native/concat.sql.out | 20 +- .../native/decimalPrecision.sql.out | 240 +- .../typeCoercion/native/division.sql.out | 68 +- .../typeCoercion/native/ifCoercion.sql.out | 162 +- .../native/implicitTypeCasts.sql.out | 12 +- .../typeCoercion/native/inConversion.sql.out | 260 +-- .../typeCoercion/native/mapZipWith.sql.out | 84 +- .../typeCoercion/native/mapconcat.sql.out | 20 +- .../native/promoteStrings.sql.out | 1162 +++++++--- .../native/stringCastAndExpressions.sql.out | 353 ++- .../native/widenSetOperationTypes.sql.out | 216 +- .../native/windowFrameCoercion.sql.out | 9 +- .../results/udaf/udaf-group-by.sql.out | 20 +- .../results/udf/udf-group-by.sql.out | 22 +- .../sql-tests/results/udf/udf-union.sql.out | 69 +- .../sql-tests/results/udf/udf-window.sql.out | 26 +- .../resources/sql-tests/results/union.sql.out | 67 +- .../sql-tests/results/window.sql.out | 26 +- .../apache/spark/sql/SQLQueryTestHelper.scala | 14 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 10 +- .../ThriftServerQueryTestSuite.scala | 10 +- 172 files changed, 13906 insertions(+), 10703 deletions(-) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/array.sql.out (93%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/cast.sql.out (82%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/conditional-functions.sql.out (99%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/date.sql.out (84%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/datetime-parsing-invalid.sql.out (71%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/datetime-special.sql.out (56%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/decimalArithmeticOperations.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/double-quoted-identifiers-disabled.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/double-quoted-identifiers-enabled.sql.out (56%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/higher-order-functions.sql.out (97%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/interval.sql.out (98%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/keywords.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/literals.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/map.sql.out (97%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/math.sql.out (94%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/parse-schema-string.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/string-functions.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/timestamp.sql.out (76%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/try_aggregates.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/try_arithmetic.sql.out (99%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/try_datetime_functions.sql.out (100%) rename sql/core/src/test/resources/sql-tests/analyzer-results/{ansi => nonansi}/try_element_at.sql.out (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/array.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/cast.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/conditional-functions.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/date.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/datetime-parsing-invalid.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/datetime-special.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/decimalArithmeticOperations.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/double-quoted-identifiers-disabled.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/double-quoted-identifiers-enabled.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/higher-order-functions.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/interval.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/keywords.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/literals.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/map.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/math.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/parse-schema-string.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/string-functions.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/timestamp.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/try_aggregates.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/try_arithmetic.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/try_datetime_functions.sql (100%) rename sql/core/src/test/resources/sql-tests/inputs/{ansi => nonansi}/try_element_at.sql (100%) delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out.java21 rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/array.sql.out (85%) create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/cast.sql.out rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/conditional-functions.sql.out (78%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/date.sql.out (91%) create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/datetime-special.sql.out (100%) create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/double-quoted-identifiers-disabled.sql.out (100%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/double-quoted-identifiers-enabled.sql.out (66%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/higher-order-functions.sql.out (100%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/interval.sql.out (94%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/keywords.sql.out (75%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/literals.sql.out (100%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/map.sql.out (100%) create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/math.sql.out rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/parse-schema-string.sql.out (100%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/string-functions.sql.out (96%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/timestamp.sql.out (89%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/try_aggregates.sql.out (67%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/try_arithmetic.sql.out (66%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/try_datetime_functions.sql.out (100%) rename sql/core/src/test/resources/sql-tests/results/{ansi => nonansi}/try_element_at.sql.out (100%) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out index 4db56d6c70561..53595d1b8a3eb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/array.sql.out @@ -212,7 +212,7 @@ select size(timestamp_array) from primitive_arrays -- !query analysis -Project [size(boolean_array#x, true) AS size(boolean_array)#x, size(tinyint_array#x, true) AS size(tinyint_array)#x, size(smallint_array#x, true) AS size(smallint_array)#x, size(int_array#x, true) AS size(int_array)#x, size(bigint_array#x, true) AS size(bigint_array)#x, size(decimal_array#x, true) AS size(decimal_array)#x, size(double_array#x, true) AS size(double_array)#x, size(float_array#x, true) AS size(float_array)#x, size(date_array#x, true) AS size(date_array)#x, size(timestamp_array#x, true) AS size(timestamp_array)#x] +Project [size(boolean_array#x, false) AS size(boolean_array)#x, size(tinyint_array#x, false) AS size(tinyint_array)#x, size(smallint_array#x, false) AS size(smallint_array)#x, size(int_array#x, false) AS size(int_array)#x, size(bigint_array#x, false) AS size(bigint_array)#x, size(decimal_array#x, false) AS size(decimal_array)#x, size(double_array#x, false) AS size(double_array)#x, size(float_array#x, false) AS size(float_array)#x, size(date_array#x, false) AS size(date_array)#x, size(timestamp_array#x, false) AS size(timestamp_array)#x] +- SubqueryAlias primitive_arrays +- View (`primitive_arrays`, [boolean_array#x, tinyint_array#x, smallint_array#x, int_array#x, bigint_array#x, decimal_array#x, double_array#x, float_array#x, date_array#x, timestamp_array#x]) +- Project [cast(boolean_array#x as array) AS boolean_array#x, cast(tinyint_array#x as array) AS tinyint_array#x, cast(smallint_array#x as array) AS smallint_array#x, cast(int_array#x as array) AS int_array#x, cast(bigint_array#x as array) AS bigint_array#x, cast(decimal_array#x as array) AS decimal_array#x, cast(double_array#x as array) AS double_array#x, cast(float_array#x as array) AS float_array#x, cast(date_array#x as array) AS date_array#x, cast(timestamp_array#x as array) AS timestamp_array#x] @@ -224,70 +224,70 @@ Project [size(boolean_array#x, true) AS size(boolean_array)#x, size(tinyint_arra -- !query select element_at(array(1, 2, 3), 5) -- !query analysis -Project [element_at(array(1, 2, 3), 5, None, false) AS element_at(array(1, 2, 3), 5)#x] +Project [element_at(array(1, 2, 3), 5, None, true) AS element_at(array(1, 2, 3), 5)#x] +- OneRowRelation -- !query select element_at(array(1, 2, 3), -5) -- !query analysis -Project [element_at(array(1, 2, 3), -5, None, false) AS element_at(array(1, 2, 3), -5)#x] +Project [element_at(array(1, 2, 3), -5, None, true) AS element_at(array(1, 2, 3), -5)#x] +- OneRowRelation -- !query select element_at(array(1, 2, 3), 0) -- !query analysis -Project [element_at(array(1, 2, 3), 0, None, false) AS element_at(array(1, 2, 3), 0)#x] +Project [element_at(array(1, 2, 3), 0, None, true) AS element_at(array(1, 2, 3), 0)#x] +- OneRowRelation -- !query select elt(4, '123', '456') -- !query analysis -Project [elt(4, 123, 456, false) AS elt(4, 123, 456)#x] +Project [elt(4, 123, 456, true) AS elt(4, 123, 456)#x] +- OneRowRelation -- !query select elt(0, '123', '456') -- !query analysis -Project [elt(0, 123, 456, false) AS elt(0, 123, 456)#x] +Project [elt(0, 123, 456, true) AS elt(0, 123, 456)#x] +- OneRowRelation -- !query select elt(-1, '123', '456') -- !query analysis -Project [elt(-1, 123, 456, false) AS elt(-1, 123, 456)#x] +Project [elt(-1, 123, 456, true) AS elt(-1, 123, 456)#x] +- OneRowRelation -- !query select elt(null, '123', '456') -- !query analysis -Project [elt(cast(null as int), 123, 456, false) AS elt(NULL, 123, 456)#x] +Project [elt(cast(null as int), 123, 456, true) AS elt(NULL, 123, 456)#x] +- OneRowRelation -- !query select elt(null, '123', null) -- !query analysis -Project [elt(cast(null as int), 123, cast(null as string), false) AS elt(NULL, 123, NULL)#x] +Project [elt(cast(null as int), 123, cast(null as string), true) AS elt(NULL, 123, NULL)#x] +- OneRowRelation -- !query select elt(1, '123', null) -- !query analysis -Project [elt(1, 123, cast(null as string), false) AS elt(1, 123, NULL)#x] +Project [elt(1, 123, cast(null as string), true) AS elt(1, 123, NULL)#x] +- OneRowRelation -- !query select elt(2, '123', null) -- !query analysis -Project [elt(2, 123, cast(null as string), false) AS elt(2, 123, NULL)#x] +Project [elt(2, 123, cast(null as string), true) AS elt(2, 123, NULL)#x] +- OneRowRelation @@ -360,21 +360,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2), true) AS size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2), false) AS size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +- OneRowRelation -- !query select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2, 3), true) AS size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2, 3), false) AS size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +- OneRowRelation -- !query select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10), 0, 1, 2, 3), true) AS size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10), 0, 1, 2, 3), false) AS size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out index e0687b564d3d1..643dfd3771ffe 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cast.sql.out @@ -205,57 +205,193 @@ Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] -- !query SELECT HEX(CAST(CAST(123 AS byte) AS binary)) -- !query analysis -Project [hex(cast(cast(123 as tinyint) as binary)) AS hex(CAST(CAST(123 AS TINYINT) AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "CAST(CAST(123 AS byte) AS binary)" + } ] +} -- !query SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) -- !query analysis -Project [hex(cast(cast(-123 as tinyint) as binary)) AS hex(CAST(CAST(-123 AS TINYINT) AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 45, + "fragment" : "CAST(CAST(-123 AS byte) AS binary)" + } ] +} -- !query SELECT HEX(CAST(123S AS binary)) -- !query analysis -Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123S AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123S AS binary)) -- !query analysis -Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123S AS binary)" + } ] +} -- !query SELECT HEX(CAST(123 AS binary)) -- !query analysis -Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 30, + "fragment" : "CAST(123 AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123 AS binary)) -- !query analysis -Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(-123 AS binary)" + } ] +} -- !query SELECT HEX(CAST(123L AS binary)) -- !query analysis -Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123L AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123L AS binary)) -- !query analysis -Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123L AS binary)" + } ] +} -- !query @@ -804,8 +940,25 @@ Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] -- !query SELECT HEX((123 :: byte) :: binary) -- !query analysis -Project [hex(cast(cast(123 as tinyint) as binary)) AS hex(CAST(CAST(123 AS TINYINT) AS BINARY))#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 34, + "fragment" : "(123 :: byte) :: binary" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index c8e28c2cfafc9..45ab1cdcff79e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -884,7 +884,7 @@ Project [concat_ws(,, cast(utf8_lcase#x as string), collate(word, utf8_binary)) -- !query select elt(2, s, utf8_binary) from t5 -- !query analysis -Project [elt(2, s#x, utf8_binary#x, false) AS elt(2, s, utf8_binary)#x] +Project [elt(2, s#x, utf8_binary#x, true) AS elt(2, s, utf8_binary)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -918,7 +918,7 @@ org.apache.spark.sql.AnalysisException -- !query select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5 -- !query analysis -Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), false) AS elt(1, collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x] +Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_binary), true) AS elt(1, collate(utf8_binary, utf8_binary), collate(utf8_lcase, utf8_binary))#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -926,7 +926,7 @@ Project [elt(1, collate(utf8_binary#x, utf8_binary), collate(utf8_lcase#x, utf8_ -- !query select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5 -- !query analysis -Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string), false) AS elt(1, collate(utf8_binary, utf8_binary), utf8_lcase)#x] +Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string), true) AS elt(1, collate(utf8_binary, utf8_binary), utf8_lcase)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -934,7 +934,7 @@ Project [elt(1, collate(utf8_binary#x, utf8_binary), cast(utf8_lcase#x as string -- !query select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 -- !query analysis -Project [elt(1, utf8_binary#x, word, false) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, cast(word as string collate UTF8_LCASE), false) AS elt(1, utf8_lcase, word)#x] +Project [elt(1, utf8_binary#x, word, true) AS elt(1, utf8_binary, word)#x, elt(1, utf8_lcase#x, cast(word as string collate UTF8_LCASE), true) AS elt(1, utf8_lcase, word)#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet @@ -942,7 +942,7 @@ Project [elt(1, utf8_binary#x, word, false) AS elt(1, utf8_binary, word)#x, elt( -- !query select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5 -- !query analysis -Project [elt(1, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase), false) AS elt(1, utf8_binary, collate(word, utf8_lcase))#x, elt(1, cast(utf8_lcase#x as string), collate(word, utf8_binary), false) AS elt(1, utf8_lcase, collate(word, utf8_binary))#x] +Project [elt(1, cast(utf8_binary#x as string collate UTF8_LCASE), collate(word, utf8_lcase), true) AS elt(1, utf8_binary, collate(word, utf8_lcase))#x, elt(1, cast(utf8_lcase#x as string), collate(word, utf8_binary), true) AS elt(1, utf8_lcase, collate(word, utf8_binary))#x] +- SubqueryAlias spark_catalog.default.t5 +- Relation spark_catalog.default.t5[s#x,utf8_binary#x,utf8_lcase#x] parquet diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/comparator.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/comparator.sql.out index 022c260ac6f60..cf17e20fc76df 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/comparator.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/comparator.sql.out @@ -16,28 +16,28 @@ Project [(0x00 < 0xFF) AS (X'00' < X'FF')#x] -- !query select '1 ' = 1Y -- !query analysis -Project [(cast(1 as tinyint) = 1) AS (1 = 1)#x] +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 1)#x] +- OneRowRelation -- !query select '\t1 ' = 1Y -- !query analysis -Project [(cast( 1 as tinyint) = 1) AS ( 1 = 1)#x] +Project [(cast( 1 as bigint) = cast(1 as bigint)) AS ( 1 = 1)#x] +- OneRowRelation -- !query select '1 ' = 1S -- !query analysis -Project [(cast(1 as smallint) = 1) AS (1 = 1)#x] +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 1)#x] +- OneRowRelation -- !query select '1 ' = 1 -- !query analysis -Project [(cast(1 as int) = 1) AS (1 = 1)#x] +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 1)#x] +- OneRowRelation @@ -51,7 +51,7 @@ Project [(cast( 1 as bigint) = 1) AS ( 1 = 1)#x] -- !query select ' 1' = cast(1.0 as float) -- !query analysis -Project [(cast( 1 as float) = cast(1.0 as float)) AS ( 1 = CAST(1.0 AS FLOAT))#x] +Project [(cast( 1 as double) = cast(cast(1.0 as float) as double)) AS ( 1 = CAST(1.0 AS FLOAT))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out index 4149f5f09947c..691864ef8b1cb 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/csv-functions.sql.out @@ -217,7 +217,7 @@ Project [to_csv(named_struct(a, 1, b, 2), Some(America/Los_Angeles)) AS to_csv(n -- !query select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) -- !query analysis -Project [to_csv((timestampFormat,dd/MM/yyyy), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false)), Some(America/Los_Angeles)) AS to_csv(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +Project [to_csv((timestampFormat,dd/MM/yyyy), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true)), Some(America/Los_Angeles)) AS to_csv(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out index 88c7d7b4e7d72..0e4d2d4e99e26 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/date.sql.out @@ -37,21 +37,21 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query select make_date(2019, 1, 1), make_date(12, 12, 12) -- !query analysis -Project [make_date(2019, 1, 1, false) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, false) AS make_date(12, 12, 12)#x] +Project [make_date(2019, 1, 1, true) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, true) AS make_date(12, 12, 12)#x] +- OneRowRelation -- !query select make_date(2000, 13, 1) -- !query analysis -Project [make_date(2000, 13, 1, false) AS make_date(2000, 13, 1)#x] +Project [make_date(2000, 13, 1, true) AS make_date(2000, 13, 1)#x] +- OneRowRelation -- !query select make_date(2000, 1, 33) -- !query analysis -Project [make_date(2000, 1, 33, false) AS make_date(2000, 1, 33)#x] +Project [make_date(2000, 1, 33, true) AS make_date(2000, 1, 33)#x] +- OneRowRelation @@ -148,21 +148,21 @@ select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(n -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_date(cast(null as string), None, Some(America/Los_Angeles), false) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), false) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), false) AS to_date(2016-12-31, yyyy-MM-dd)#x] +Project [to_date(cast(null as string), None, Some(America/Los_Angeles), true) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), true) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), true) AS to_date(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_date("16", "dd") -- !query analysis -Project [to_date(16, Some(dd), Some(America/Los_Angeles), false) AS to_date(16, dd)#x] +Project [to_date(16, Some(dd), Some(America/Los_Angeles), true) AS to_date(16, dd)#x] +- OneRowRelation -- !query select to_date("02-29", "MM-dd") -- !query analysis -Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), false) AS to_date(02-29, MM-dd)#x] +Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), true) AS to_date(02-29, MM-dd)#x] +- OneRowRelation @@ -201,21 +201,21 @@ select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(time -- !query select next_day("2015-07-23", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 as date), Mon, false) AS next_day(2015-07-23, Mon)#x] +Project [next_day(cast(2015-07-23 as date), Mon, true) AS next_day(2015-07-23, Mon)#x] +- OneRowRelation -- !query select next_day("2015-07-23", "xx") -- !query analysis -Project [next_day(cast(2015-07-23 as date), xx, false) AS next_day(2015-07-23, xx)#x] +Project [next_day(cast(2015-07-23 as date), xx, true) AS next_day(2015-07-23, xx)#x] +- OneRowRelation -- !query select next_day("2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(2015-07-23 12:12:12, Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(2015-07-23 12:12:12, Mon)#x] +- OneRowRelation @@ -228,28 +228,28 @@ select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") -- !query select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +- OneRowRelation -- !query select next_day("xx", "Mon") -- !query analysis -Project [next_day(cast(xx as date), Mon, false) AS next_day(xx, Mon)#x] +Project [next_day(cast(xx as date), Mon, true) AS next_day(xx, Mon)#x] +- OneRowRelation -- !query select next_day(null, "Mon") -- !query analysis -Project [next_day(cast(null as date), Mon, false) AS next_day(NULL, Mon)#x] +Project [next_day(cast(null as date), Mon, true) AS next_day(NULL, Mon)#x] +- OneRowRelation -- !query select next_day(null, "xx") -- !query analysis -Project [next_day(cast(null as date), xx, false) AS next_day(NULL, xx)#x] +Project [next_day(cast(null as date), xx, true) AS next_day(NULL, xx)#x] +- OneRowRelation @@ -355,21 +355,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select date_add('2011-11-11', '1') -- !query analysis -Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] +Project [date_add(cast(2011-11-11 as date), cast(1 as int)) AS date_add(2011-11-11, 1)#x] +- OneRowRelation -- !query select date_add('2011-11-11', '1.2') -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", - "messageParameters" : { - "functionName" : "date_add" - } -} +Project [date_add(cast(2011-11-11 as date), cast(1.2 as int)) AS date_add(2011-11-11, 1.2)#x] ++- OneRowRelation -- !query @@ -505,14 +499,7 @@ select date_sub(date'2011-11-11', '1') -- !query select date_sub(date'2011-11-11', '1.2') -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", - "messageParameters" : { - "functionName" : "date_sub" - } -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -543,49 +530,23 @@ Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NT -- !query select date_add('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_add('2011-11-11', int_str)" - } ] -} +Project [date_add(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_add(2011-11-11, int_str)#x] ++- SubqueryAlias date_view + +- View (`date_view`, [date_str#x, int_str#x]) + +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation -- !query select date_sub('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_sub('2011-11-11', int_str)" - } ] -} +Project [date_sub(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_sub(2011-11-11, int_str)#x] ++- SubqueryAlias date_view + +- View (`date_view`, [date_str#x, int_str#x]) + +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation -- !query @@ -661,25 +622,7 @@ select date '2001-10-01' - date '2001-09-28' -- !query select date '2001-10-01' - '2001-09-28' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2001-09-28\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "date '2001-10-01' - '2001-09-28'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -709,25 +652,7 @@ select date_str - date '2001-09-28' from date_view -- !query select date '2001-09-28' - date_str from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"date_str\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "date '2001-09-28' - date_str" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -739,7 +664,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -762,11 +687,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" }, "queryContext" : [ { "objectType" : "", @@ -804,7 +729,7 @@ select date '2012-01-01' - interval '2-2' year to month, -- !query select to_date('26/October/2015', 'dd/MMMMM/yyyy') -- !query analysis -Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), false) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), true) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out index 4221db822d024..3681a5dfd3904 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-legacy.sql.out @@ -37,21 +37,21 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query select make_date(2019, 1, 1), make_date(12, 12, 12) -- !query analysis -Project [make_date(2019, 1, 1, false) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, false) AS make_date(12, 12, 12)#x] +Project [make_date(2019, 1, 1, true) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, true) AS make_date(12, 12, 12)#x] +- OneRowRelation -- !query select make_date(2000, 13, 1) -- !query analysis -Project [make_date(2000, 13, 1, false) AS make_date(2000, 13, 1)#x] +Project [make_date(2000, 13, 1, true) AS make_date(2000, 13, 1)#x] +- OneRowRelation -- !query select make_date(2000, 1, 33) -- !query analysis -Project [make_date(2000, 1, 33, false) AS make_date(2000, 1, 33)#x] +Project [make_date(2000, 1, 33, true) AS make_date(2000, 1, 33)#x] +- OneRowRelation @@ -148,21 +148,21 @@ select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(n -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_date(cast(null as string), None, Some(America/Los_Angeles), false) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), false) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), false) AS to_date(2016-12-31, yyyy-MM-dd)#x] +Project [to_date(cast(null as string), None, Some(America/Los_Angeles), true) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), true) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), true) AS to_date(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_date("16", "dd") -- !query analysis -Project [to_date(16, Some(dd), Some(America/Los_Angeles), false) AS to_date(16, dd)#x] +Project [to_date(16, Some(dd), Some(America/Los_Angeles), true) AS to_date(16, dd)#x] +- OneRowRelation -- !query select to_date("02-29", "MM-dd") -- !query analysis -Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), false) AS to_date(02-29, MM-dd)#x] +Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), true) AS to_date(02-29, MM-dd)#x] +- OneRowRelation @@ -201,21 +201,21 @@ select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(time -- !query select next_day("2015-07-23", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 as date), Mon, false) AS next_day(2015-07-23, Mon)#x] +Project [next_day(cast(2015-07-23 as date), Mon, true) AS next_day(2015-07-23, Mon)#x] +- OneRowRelation -- !query select next_day("2015-07-23", "xx") -- !query analysis -Project [next_day(cast(2015-07-23 as date), xx, false) AS next_day(2015-07-23, xx)#x] +Project [next_day(cast(2015-07-23 as date), xx, true) AS next_day(2015-07-23, xx)#x] +- OneRowRelation -- !query select next_day("2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(2015-07-23 12:12:12, Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(2015-07-23 12:12:12, Mon)#x] +- OneRowRelation @@ -228,28 +228,28 @@ select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") -- !query select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +- OneRowRelation -- !query select next_day("xx", "Mon") -- !query analysis -Project [next_day(cast(xx as date), Mon, false) AS next_day(xx, Mon)#x] +Project [next_day(cast(xx as date), Mon, true) AS next_day(xx, Mon)#x] +- OneRowRelation -- !query select next_day(null, "Mon") -- !query analysis -Project [next_day(cast(null as date), Mon, false) AS next_day(NULL, Mon)#x] +Project [next_day(cast(null as date), Mon, true) AS next_day(NULL, Mon)#x] +- OneRowRelation -- !query select next_day(null, "xx") -- !query analysis -Project [next_day(cast(null as date), xx, false) AS next_day(NULL, xx)#x] +Project [next_day(cast(null as date), xx, true) AS next_day(NULL, xx)#x] +- OneRowRelation @@ -355,21 +355,15 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select date_add('2011-11-11', '1') -- !query analysis -Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] +Project [date_add(cast(2011-11-11 as date), cast(1 as int)) AS date_add(2011-11-11, 1)#x] +- OneRowRelation -- !query select date_add('2011-11-11', '1.2') -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", - "messageParameters" : { - "functionName" : "date_add" - } -} +Project [date_add(cast(2011-11-11 as date), cast(1.2 as int)) AS date_add(2011-11-11, 1.2)#x] ++- OneRowRelation -- !query @@ -505,14 +499,7 @@ select date_sub(date'2011-11-11', '1') -- !query select date_sub(date'2011-11-11', '1.2') -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", - "messageParameters" : { - "functionName" : "date_sub" - } -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -543,49 +530,23 @@ Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NT -- !query select date_add('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_add('2011-11-11', int_str)" - } ] -} +Project [date_add(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_add(2011-11-11, int_str)#x] ++- SubqueryAlias date_view + +- View (`date_view`, [date_str#x, int_str#x]) + +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation -- !query select date_sub('2011-11-11', int_str) from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_sub('2011-11-11', int_str)" - } ] -} +Project [date_sub(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_sub(2011-11-11, int_str)#x] ++- SubqueryAlias date_view + +- View (`date_view`, [date_str#x, int_str#x]) + +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] + +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] + +- OneRowRelation -- !query @@ -661,25 +622,7 @@ select date '2001-10-01' - date '2001-09-28' -- !query select date '2001-10-01' - '2001-09-28' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2001-09-28\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "date '2001-10-01' - '2001-09-28'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -709,25 +652,7 @@ select date_str - date '2001-09-28' from date_view -- !query select date '2001-09-28' - date_str from date_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"date_str\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "date '2001-09-28' - date_str" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -739,7 +664,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -762,11 +687,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" }, "queryContext" : [ { "objectType" : "", @@ -804,7 +729,7 @@ select date '2012-01-01' - interval '2-2' year to month, -- !query select to_date('26/October/2015', 'dd/MMMMM/yyyy') -- !query analysis -Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), false) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), true) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +- OneRowRelation @@ -1121,70 +1046,70 @@ Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(Americ -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 1) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 60) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, null) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +- OneRowRelation @@ -1333,231 +1258,231 @@ select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020- -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp(1) -- !query analysis -Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1)#x] +Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1)#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") -- !query analysis -Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +- OneRowRelation -- !query select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") -- !query analysis -Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +- OneRowRelation -- !query select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") -- !query analysis -Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +- OneRowRelation -- !query select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("16", "dd") -- !query analysis -Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(16, dd)#x] +Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(16, dd)#x] +- OneRowRelation -- !query select to_timestamp("02-29", "MM-dd") -- !query analysis -Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(02-29, MM-dd)#x] +Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(02-29, MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019 40", "yyyy mm") -- !query analysis -Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 40, yyyy mm)#x] +Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 40, yyyy mm)#x] +- OneRowRelation -- !query select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") -- !query analysis -Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +- OneRowRelation @@ -1582,49 +1507,13 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:10\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:11\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -1650,49 +1539,13 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "str - timestamp'2011-11-11 11:11:11'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "timestamp'2011-11-11 11:11:11' - str" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -1700,11 +1553,11 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -1722,11 +1575,11 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { @@ -1805,28 +1658,28 @@ select date '2012-01-01' - interval 3 hours, -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query analysis -Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +- OneRowRelation -- !query select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), true) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-invalid.sql.out index ad4a2feb9661c..74146ab17a4d4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-invalid.sql.out @@ -2,126 +2,126 @@ -- !query select to_timestamp('294248', 'y') -- !query analysis -Project [to_timestamp(294248, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(294248, y)#x] +Project [to_timestamp(294248, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(294248, y)#x] +- OneRowRelation -- !query select to_timestamp('1', 'yy') -- !query analysis -Project [to_timestamp(1, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, yy)#x] +Project [to_timestamp(1, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, yy)#x] +- OneRowRelation -- !query select to_timestamp('-12', 'yy') -- !query analysis -Project [to_timestamp(-12, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(-12, yy)#x] +Project [to_timestamp(-12, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(-12, yy)#x] +- OneRowRelation -- !query select to_timestamp('123', 'yy') -- !query analysis -Project [to_timestamp(123, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123, yy)#x] +Project [to_timestamp(123, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(123, yy)#x] +- OneRowRelation -- !query select to_timestamp('1', 'yyy') -- !query analysis -Project [to_timestamp(1, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, yyy)#x] +Project [to_timestamp(1, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, yyy)#x] +- OneRowRelation -- !query select to_timestamp('1234567', 'yyyyyyy') -- !query analysis -Project [to_timestamp(1234567, Some(yyyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1234567, yyyyyyy)#x] +Project [to_timestamp(1234567, Some(yyyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1234567, yyyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('366', 'D') -- !query analysis -Project [to_timestamp(366, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(366, D)#x] +Project [to_timestamp(366, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(366, D)#x] +- OneRowRelation -- !query select to_timestamp('9', 'DD') -- !query analysis -Project [to_timestamp(9, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, DD)#x] +Project [to_timestamp(9, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, DD)#x] +- OneRowRelation -- !query select to_timestamp('366', 'DD') -- !query analysis -Project [to_timestamp(366, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(366, DD)#x] +Project [to_timestamp(366, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(366, DD)#x] +- OneRowRelation -- !query select to_timestamp('9', 'DDD') -- !query analysis -Project [to_timestamp(9, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, DDD)#x] +Project [to_timestamp(9, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, DDD)#x] +- OneRowRelation -- !query select to_timestamp('99', 'DDD') -- !query analysis -Project [to_timestamp(99, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DDD)#x] +Project [to_timestamp(99, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, DDD)#x] +- OneRowRelation -- !query select to_timestamp('30-365', 'dd-DDD') -- !query analysis -Project [to_timestamp(30-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(30-365, dd-DDD)#x] +Project [to_timestamp(30-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(30-365, dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('11-365', 'MM-DDD') -- !query analysis -Project [to_timestamp(11-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(11-365, MM-DDD)#x] +Project [to_timestamp(11-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(11-365, MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2019-366', 'yyyy-DDD') -- !query analysis -Project [to_timestamp(2019-366, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-366, yyyy-DDD)#x] +Project [to_timestamp(2019-366, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-366, yyyy-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-30-365', 'MM-dd-DDD') -- !query analysis -Project [to_timestamp(12-30-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-30-365, MM-dd-DDD)#x] +Project [to_timestamp(12-30-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-30-365, MM-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-01-365', 'yyyy-dd-DDD') -- !query analysis -Project [to_timestamp(2020-01-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-01-365, yyyy-dd-DDD)#x] +Project [to_timestamp(2020-01-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-01-365, yyyy-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-10-350', 'yyyy-MM-DDD') -- !query analysis -Project [to_timestamp(2020-10-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-10-350, yyyy-MM-DDD)#x] +Project [to_timestamp(2020-10-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-10-350, yyyy-MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') -- !query analysis -Project [to_timestamp(2020-11-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-11-31-366, yyyy-MM-dd-DDD)#x] +Project [to_timestamp(2020-11-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-11-31-366, yyyy-MM-dd-DDD)#x] +- OneRowRelation @@ -135,56 +135,56 @@ Project [from_csv(StructField(date,DateType,true), (dateFormat,yyyy-DDD), 2018-3 -- !query select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_date(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), false) AS to_date(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_date(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), true) AS to_date(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_date(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), false) AS to_date(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_date(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), true) AS to_date(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_timestamp(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_timestamp(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_timestamp(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_timestamp(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-legacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-legacy.sql.out index 7325f2756949e..7907279fb0204 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing-legacy.sql.out @@ -2,180 +2,180 @@ -- !query select to_timestamp('1', 'y') -- !query analysis -Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, y)#x] +Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, y)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'y') -- !query analysis -Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, y)#x] +Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, y)#x] +- OneRowRelation -- !query select to_timestamp('00', 'yy') -- !query analysis -Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00, yy)#x] +Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(00, yy)#x] +- OneRowRelation -- !query select to_timestamp('99', 'yy') -- !query analysis -Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, yy)#x] +Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, yy)#x] +- OneRowRelation -- !query select to_timestamp('001', 'yyy') -- !query analysis -Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(001, yyy)#x] +Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(001, yyy)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'yyy') -- !query analysis -Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyy)#x] +Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, yyy)#x] +- OneRowRelation -- !query select to_timestamp('0001', 'yyyy') -- !query analysis -Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(0001, yyyy)#x] +Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(0001, yyyy)#x] +- OneRowRelation -- !query select to_timestamp('9999', 'yyyy') -- !query analysis -Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9999, yyyy)#x] +Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9999, yyyy)#x] +- OneRowRelation -- !query select to_timestamp('00001', 'yyyyy') -- !query analysis -Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00001, yyyyy)#x] +Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(00001, yyyyy)#x] +- OneRowRelation -- !query select to_timestamp('09999', 'yyyyy') -- !query analysis -Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09999, yyyyy)#x] +Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(09999, yyyyy)#x] +- OneRowRelation -- !query select to_timestamp('000001', 'yyyyyy') -- !query analysis -Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(000001, yyyyyy)#x] +Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(000001, yyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'yyyyyy') -- !query analysis -Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyyyyy)#x] +Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, yyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('9', 'D') -- !query analysis -Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, D)#x] +Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, D)#x] +- OneRowRelation -- !query select to_timestamp('300', 'D') -- !query analysis -Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(300, D)#x] +Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(300, D)#x] +- OneRowRelation -- !query select to_timestamp('09', 'DD') -- !query analysis -Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09, DD)#x] +Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(09, DD)#x] +- OneRowRelation -- !query select to_timestamp('99', 'DD') -- !query analysis -Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DD)#x] +Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, DD)#x] +- OneRowRelation -- !query select to_timestamp('100', 'DD') -- !query analysis -Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(100, DD)#x] +Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(100, DD)#x] +- OneRowRelation -- !query select to_timestamp('009', 'DDD') -- !query analysis -Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009, DDD)#x] +Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009, DDD)#x] +- OneRowRelation -- !query select to_timestamp('365', 'DDD') -- !query analysis -Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(365, DDD)#x] +Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(365, DDD)#x] +- OneRowRelation -- !query select to_timestamp('31-365', 'dd-DDD') -- !query analysis -Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(31-365, dd-DDD)#x] +Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(31-365, dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-365', 'MM-DDD') -- !query analysis -Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-365, MM-DDD)#x] +Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-365, MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-365', 'yyyy-DDD') -- !query analysis -Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-365, yyyy-DDD)#x] +Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-365, yyyy-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-31-365', 'MM-dd-DDD') -- !query analysis -Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-31-365, MM-dd-DDD)#x] +Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-31-365, MM-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-30-365', 'yyyy-dd-DDD') -- !query analysis -Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x] +Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-12-350', 'yyyy-MM-DDD') -- !query analysis -Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x] +Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') -- !query analysis -Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x] +Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing.sql.out index 7325f2756949e..7907279fb0204 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-parsing.sql.out @@ -2,180 +2,180 @@ -- !query select to_timestamp('1', 'y') -- !query analysis -Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, y)#x] +Project [to_timestamp(1, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, y)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'y') -- !query analysis -Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, y)#x] +Project [to_timestamp(009999, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, y)#x] +- OneRowRelation -- !query select to_timestamp('00', 'yy') -- !query analysis -Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00, yy)#x] +Project [to_timestamp(00, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(00, yy)#x] +- OneRowRelation -- !query select to_timestamp('99', 'yy') -- !query analysis -Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, yy)#x] +Project [to_timestamp(99, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, yy)#x] +- OneRowRelation -- !query select to_timestamp('001', 'yyy') -- !query analysis -Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(001, yyy)#x] +Project [to_timestamp(001, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(001, yyy)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'yyy') -- !query analysis -Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyy)#x] +Project [to_timestamp(009999, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, yyy)#x] +- OneRowRelation -- !query select to_timestamp('0001', 'yyyy') -- !query analysis -Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(0001, yyyy)#x] +Project [to_timestamp(0001, Some(yyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(0001, yyyy)#x] +- OneRowRelation -- !query select to_timestamp('9999', 'yyyy') -- !query analysis -Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9999, yyyy)#x] +Project [to_timestamp(9999, Some(yyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9999, yyyy)#x] +- OneRowRelation -- !query select to_timestamp('00001', 'yyyyy') -- !query analysis -Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(00001, yyyyy)#x] +Project [to_timestamp(00001, Some(yyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(00001, yyyyy)#x] +- OneRowRelation -- !query select to_timestamp('09999', 'yyyyy') -- !query analysis -Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09999, yyyyy)#x] +Project [to_timestamp(09999, Some(yyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(09999, yyyyy)#x] +- OneRowRelation -- !query select to_timestamp('000001', 'yyyyyy') -- !query analysis -Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(000001, yyyyyy)#x] +Project [to_timestamp(000001, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(000001, yyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('009999', 'yyyyyy') -- !query analysis -Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009999, yyyyyy)#x] +Project [to_timestamp(009999, Some(yyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009999, yyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('9', 'D') -- !query analysis -Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, D)#x] +Project [to_timestamp(9, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, D)#x] +- OneRowRelation -- !query select to_timestamp('300', 'D') -- !query analysis -Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(300, D)#x] +Project [to_timestamp(300, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(300, D)#x] +- OneRowRelation -- !query select to_timestamp('09', 'DD') -- !query analysis -Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(09, DD)#x] +Project [to_timestamp(09, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(09, DD)#x] +- OneRowRelation -- !query select to_timestamp('99', 'DD') -- !query analysis -Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DD)#x] +Project [to_timestamp(99, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, DD)#x] +- OneRowRelation -- !query select to_timestamp('100', 'DD') -- !query analysis -Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(100, DD)#x] +Project [to_timestamp(100, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(100, DD)#x] +- OneRowRelation -- !query select to_timestamp('009', 'DDD') -- !query analysis -Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(009, DDD)#x] +Project [to_timestamp(009, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(009, DDD)#x] +- OneRowRelation -- !query select to_timestamp('365', 'DDD') -- !query analysis -Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(365, DDD)#x] +Project [to_timestamp(365, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(365, DDD)#x] +- OneRowRelation -- !query select to_timestamp('31-365', 'dd-DDD') -- !query analysis -Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(31-365, dd-DDD)#x] +Project [to_timestamp(31-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(31-365, dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-365', 'MM-DDD') -- !query analysis -Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-365, MM-DDD)#x] +Project [to_timestamp(12-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-365, MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-365', 'yyyy-DDD') -- !query analysis -Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-365, yyyy-DDD)#x] +Project [to_timestamp(2020-365, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-365, yyyy-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-31-365', 'MM-dd-DDD') -- !query analysis -Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-31-365, MM-dd-DDD)#x] +Project [to_timestamp(12-31-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-31-365, MM-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-30-365', 'yyyy-dd-DDD') -- !query analysis -Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x] +Project [to_timestamp(2020-30-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-30-365, yyyy-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-12-350', 'yyyy-MM-DDD') -- !query analysis -Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x] +Project [to_timestamp(2020-12-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-12-350, yyyy-MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') -- !query analysis -Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x] +Project [to_timestamp(2020-12-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-12-31-366, yyyy-MM-dd-DDD)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-special.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-special.sql.out index 01d1f2c40a4a6..6768297fd8116 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-special.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/datetime-special.sql.out @@ -8,7 +8,7 @@ select date'999999-03-18', date'-0001-1-28', date'0015' -- !query select make_date(999999, 3, 18), make_date(-1, 1, 28) -- !query analysis -Project [make_date(999999, 3, 18, false) AS make_date(999999, 3, 18)#x, make_date(-1, 1, 28, false) AS make_date(-1, 1, 28)#x] +Project [make_date(999999, 3, 18, true) AS make_date(999999, 3, 18)#x, make_date(-1, 1, 28, true) AS make_date(-1, 1, 28)#x] +- OneRowRelation @@ -21,5 +21,5 @@ select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timesta -- !query select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) -- !query analysis -Project [make_timestamp(-1969, 12, 31, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(-1969, 12, 31, 16, 0, 0.0)#x, make_timestamp(-15, 3, 18, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(-15, 3, 18, 16, 0, 0.0)#x, make_timestamp(99999, 3, 18, 12, 3, cast(17.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(99999, 3, 18, 12, 3, 17.0)#x] +Project [make_timestamp(-1969, 12, 31, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(-1969, 12, 31, 16, 0, 0.0)#x, make_timestamp(-15, 3, 18, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(-15, 3, 18, 16, 0, 0.0)#x, make_timestamp(99999, 3, 18, 12, 3, cast(17.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(99999, 3, 18, 12, 3, 17.0)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out index 4a2199033f819..d75f4d41bd425 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out @@ -268,12 +268,13 @@ SetCommand (spark.sql.decimalOperations.allowPrecisionLoss,Some(false)) -- !query -select id, a+b, a-b, a*b, a/b from decimals_test order by id +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query analysis Sort [id#x ASC NULLS FIRST], true -+- Project [id#x, (a#x + b#x) AS (a + b)#x, (a#x - b#x) AS (a - b)#x, (a#x * b#x) AS (a * b)#x, (a#x / b#x) AS (a / b)#x] - +- SubqueryAlias spark_catalog.default.decimals_test - +- Relation spark_catalog.default.decimals_test[id#x,a#x,b#x] parquet ++- Repartition 1, false + +- Project [id#x, (a#x + b#x) AS (a + b)#x, (a#x - b#x) AS (a - b)#x, (a#x * b#x) AS (a * b)#x, (a#x / b#x) AS (a / b)#x] + +- SubqueryAlias spark_catalog.default.decimals_test + +- Relation spark_catalog.default.decimals_test[id#x,a#x,b#x] parquet -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/extract.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/extract.sql.out index c42f2db3f0f9f..987941eee05c7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/extract.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/extract.sql.out @@ -3,7 +3,7 @@ CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, to_timestamp_ntz('2011-05-06 07:08:09.1234567') as ntz, interval 10 year 20 month as i, interval 30 day 40 hour 50 minute 6.7890 second as j -- !query analysis CreateViewCommand `t`, select '2011-05-06 07:08:09.1234567' as c, to_timestamp_ntz('2011-05-06 07:08:09.1234567') as ntz, interval 10 year 20 month as i, interval 30 day 40 hour 50 minute 6.7890 second as j, false, false, LocalTempView, UNSUPPORTED, true - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -14,7 +14,7 @@ Project [extract(year, c#x) AS extract(year FROM c)#x, extract(year, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -25,7 +25,7 @@ Project [extract(y, c#x) AS extract(y FROM c)#x, extract(y, ntz#x) AS extract(y +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -36,7 +36,7 @@ Project [extract(years, c#x) AS extract(years FROM c)#x, extract(years, ntz#x) A +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -47,7 +47,7 @@ Project [extract(yr, c#x) AS extract(yr FROM c)#x, extract(yr, ntz#x) AS extract +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -58,7 +58,7 @@ Project [extract(yrs, c#x) AS extract(yrs FROM c)#x, extract(yrs, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -69,7 +69,7 @@ Project [extract(yearofweek, c#x) AS extract(yearofweek FROM c)#x, extract(yearo +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -80,7 +80,7 @@ Project [extract(quarter, c#x) AS extract(quarter FROM c)#x, extract(quarter, nt +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -91,7 +91,7 @@ Project [extract(qtr, c#x) AS extract(qtr FROM c)#x, extract(qtr, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -102,7 +102,7 @@ Project [extract(month, c#x) AS extract(month FROM c)#x, extract(month, ntz#x) A +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -113,7 +113,7 @@ Project [extract(mon, c#x) AS extract(mon FROM c)#x, extract(mon, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -124,7 +124,7 @@ Project [extract(mons, c#x) AS extract(mons FROM c)#x, extract(mons, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -135,7 +135,7 @@ Project [extract(months, c#x) AS extract(months FROM c)#x, extract(months, ntz#x +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -146,7 +146,7 @@ Project [extract(week, c#x) AS extract(week FROM c)#x, extract(week, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -157,7 +157,7 @@ Project [extract(w, c#x) AS extract(w FROM c)#x, extract(w, ntz#x) AS extract(w +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -168,7 +168,7 @@ Project [extract(weeks, c#x) AS extract(weeks FROM c)#x, extract(weeks, ntz#x) A +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -179,7 +179,7 @@ Project [extract(day, c#x) AS extract(day FROM c)#x, extract(day, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -190,7 +190,7 @@ Project [extract(d, c#x) AS extract(d FROM c)#x, extract(d, ntz#x) AS extract(d +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -201,7 +201,7 @@ Project [extract(days, c#x) AS extract(days FROM c)#x, extract(days, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -212,7 +212,7 @@ Project [extract(dayofweek, c#x) AS extract(dayofweek FROM c)#x, extract(dayofwe +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -223,7 +223,7 @@ Project [extract(dow, c#x) AS extract(dow FROM c)#x, extract(dow, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -234,7 +234,7 @@ Project [extract(dayofweek_iso, c#x) AS extract(dayofweek_iso FROM c)#x, extract +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -245,7 +245,7 @@ Project [extract(dow_iso, c#x) AS extract(dow_iso FROM c)#x, extract(dow_iso, nt +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -256,7 +256,7 @@ Project [extract(doy, c#x) AS extract(doy FROM c)#x, extract(doy, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -267,7 +267,7 @@ Project [extract(hour, c#x) AS extract(hour FROM c)#x, extract(hour, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -278,7 +278,7 @@ Project [extract(h, c#x) AS extract(h FROM c)#x, extract(h, ntz#x) AS extract(h +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -289,7 +289,7 @@ Project [extract(hours, c#x) AS extract(hours FROM c)#x, extract(hours, ntz#x) A +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -300,7 +300,7 @@ Project [extract(hr, c#x) AS extract(hr FROM c)#x, extract(hr, ntz#x) AS extract +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -311,7 +311,7 @@ Project [extract(hrs, c#x) AS extract(hrs FROM c)#x, extract(hrs, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -322,7 +322,7 @@ Project [extract(minute, c#x) AS extract(minute FROM c)#x, extract(minute, ntz#x +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -333,7 +333,7 @@ Project [extract(m, c#x) AS extract(m FROM c)#x, extract(m, ntz#x) AS extract(m +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -344,7 +344,7 @@ Project [extract(min, c#x) AS extract(min FROM c)#x, extract(min, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -355,7 +355,7 @@ Project [extract(mins, c#x) AS extract(mins FROM c)#x, extract(mins, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -366,7 +366,7 @@ Project [extract(minutes, c#x) AS extract(minutes FROM c)#x, extract(minutes, nt +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -377,7 +377,7 @@ Project [extract(second, c#x) AS extract(second FROM c)#x, extract(second, ntz#x +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -388,7 +388,7 @@ Project [extract(s, c#x) AS extract(s FROM c)#x, extract(s, ntz#x) AS extract(s +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -399,7 +399,7 @@ Project [extract(sec, c#x) AS extract(sec FROM c)#x, extract(sec, ntz#x) AS extr +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -410,7 +410,7 @@ Project [extract(seconds, c#x) AS extract(seconds FROM c)#x, extract(seconds, nt +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -421,7 +421,7 @@ Project [extract(secs, c#x) AS extract(secs FROM c)#x, extract(secs, ntz#x) AS e +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -495,7 +495,7 @@ Project [date_part(year, c#x) AS date_part(year, c)#x, date_part(year, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -506,7 +506,7 @@ Project [date_part(y, c#x) AS date_part(y, c)#x, date_part(y, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -517,7 +517,7 @@ Project [date_part(years, c#x) AS date_part(years, c)#x, date_part(years, ntz#x) +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -528,7 +528,7 @@ Project [date_part(yr, c#x) AS date_part(yr, c)#x, date_part(yr, ntz#x) AS date_ +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -539,7 +539,7 @@ Project [date_part(yrs, c#x) AS date_part(yrs, c)#x, date_part(yrs, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -550,7 +550,7 @@ Project [date_part(yearofweek, c#x) AS date_part(yearofweek, c)#x, date_part(yea +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -561,7 +561,7 @@ Project [date_part(quarter, c#x) AS date_part(quarter, c)#x, date_part(quarter, +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -572,7 +572,7 @@ Project [date_part(qtr, c#x) AS date_part(qtr, c)#x, date_part(qtr, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -583,7 +583,7 @@ Project [date_part(month, c#x) AS date_part(month, c)#x, date_part(month, ntz#x) +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -594,7 +594,7 @@ Project [date_part(mon, c#x) AS date_part(mon, c)#x, date_part(mon, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -605,7 +605,7 @@ Project [date_part(mons, c#x) AS date_part(mons, c)#x, date_part(mons, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -616,7 +616,7 @@ Project [date_part(months, c#x) AS date_part(months, c)#x, date_part(months, ntz +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -627,7 +627,7 @@ Project [date_part(week, c#x) AS date_part(week, c)#x, date_part(week, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -638,7 +638,7 @@ Project [date_part(w, c#x) AS date_part(w, c)#x, date_part(w, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -649,7 +649,7 @@ Project [date_part(weeks, c#x) AS date_part(weeks, c)#x, date_part(weeks, ntz#x) +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -660,7 +660,7 @@ Project [date_part(day, c#x) AS date_part(day, c)#x, date_part(day, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -671,7 +671,7 @@ Project [date_part(d, c#x) AS date_part(d, c)#x, date_part(d, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -682,7 +682,7 @@ Project [date_part(days, c#x) AS date_part(days, c)#x, date_part(days, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -693,7 +693,7 @@ Project [date_part(dayofweek, c#x) AS date_part(dayofweek, c)#x, date_part(dayof +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -704,7 +704,7 @@ Project [date_part(dow, c#x) AS date_part(dow, c)#x, date_part(dow, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -715,7 +715,7 @@ Project [date_part(dayofweek_iso, c#x) AS date_part(dayofweek_iso, c)#x, date_pa +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -726,7 +726,7 @@ Project [date_part(dow_iso, c#x) AS date_part(dow_iso, c)#x, date_part(dow_iso, +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -737,7 +737,7 @@ Project [date_part(doy, c#x) AS date_part(doy, c)#x, date_part(doy, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -748,7 +748,7 @@ Project [date_part(hour, c#x) AS date_part(hour, c)#x, date_part(hour, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -759,7 +759,7 @@ Project [date_part(h, c#x) AS date_part(h, c)#x, date_part(h, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -770,7 +770,7 @@ Project [date_part(hours, c#x) AS date_part(hours, c)#x, date_part(hours, ntz#x) +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -781,7 +781,7 @@ Project [date_part(hr, c#x) AS date_part(hr, c)#x, date_part(hr, ntz#x) AS date_ +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -792,7 +792,7 @@ Project [date_part(hrs, c#x) AS date_part(hrs, c)#x, date_part(hrs, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -803,7 +803,7 @@ Project [date_part(minute, c#x) AS date_part(minute, c)#x, date_part(minute, ntz +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -814,7 +814,7 @@ Project [date_part(m, c#x) AS date_part(m, c)#x, date_part(m, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -825,7 +825,7 @@ Project [date_part(min, c#x) AS date_part(min, c)#x, date_part(min, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -836,7 +836,7 @@ Project [date_part(mins, c#x) AS date_part(mins, c)#x, date_part(mins, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -847,7 +847,7 @@ Project [date_part(minutes, c#x) AS date_part(minutes, c)#x, date_part(minutes, +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -858,7 +858,7 @@ Project [date_part(second, c#x) AS date_part(second, c)#x, date_part(second, ntz +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -869,7 +869,7 @@ Project [date_part(s, c#x) AS date_part(s, c)#x, date_part(s, ntz#x) AS date_par +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -880,7 +880,7 @@ Project [date_part(sec, c#x) AS date_part(sec, c)#x, date_part(sec, ntz#x) AS da +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -891,7 +891,7 @@ Project [date_part(seconds, c#x) AS date_part(seconds, c)#x, date_part(seconds, +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -902,7 +902,7 @@ Project [date_part(secs, c#x) AS date_part(secs, c)#x, date_part(secs, ntz#x) AS +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -956,7 +956,7 @@ Project [date_part(null, c#x) AS date_part(NULL, c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -989,7 +989,7 @@ Project [date_part(null, i#x) AS date_part(NULL, i)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1000,7 +1000,7 @@ Project [extract(year, c#x) AS extract(year FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1011,7 +1011,7 @@ Project [extract(quarter, c#x) AS extract(quarter FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1022,7 +1022,7 @@ Project [extract(month, c#x) AS extract(month FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1033,7 +1033,7 @@ Project [extract(week, c#x) AS extract(week FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1044,7 +1044,7 @@ Project [extract(day, c#x) AS extract(day FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1055,7 +1055,7 @@ Project [extract(days, c#x) AS extract(days FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1066,7 +1066,7 @@ Project [extract(dayofweek, c#x) AS extract(dayofweek FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1077,7 +1077,7 @@ Project [extract(dow, c#x) AS extract(dow FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1088,7 +1088,7 @@ Project [extract(doy, c#x) AS extract(doy FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1099,7 +1099,7 @@ Project [extract(hour, c#x) AS extract(hour FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1110,7 +1110,7 @@ Project [extract(minute, c#x) AS extract(minute FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1121,7 +1121,7 @@ Project [extract(second, c#x) AS extract(second FROM c)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1132,7 +1132,7 @@ Project [cast(c#x - j#x as string) AS c - j#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1143,7 +1143,7 @@ Project [day(cast(cast(c#x - j#x as string) as date)) AS day(c - j)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1154,29 +1154,29 @@ Project [extract(day, cast(c#x - j#x as string)) AS extract(day FROM c - j)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation -- !query select extract(month from to_timestamp(c) - i) from t -- !query analysis -Project [extract(month, to_timestamp(c#x, None, TimestampType, Some(America/Los_Angeles), false) - i#x) AS extract(month FROM to_timestamp(c) - i)#x] +Project [extract(month, to_timestamp(c#x, None, TimestampType, Some(America/Los_Angeles), true) - i#x) AS extract(month FROM to_timestamp(c) - i)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation -- !query select extract(second from to_timestamp(c) - j) from t -- !query analysis -Project [extract(second, cast(to_timestamp(c#x, None, TimestampType, Some(America/Los_Angeles), false) - j#x as timestamp)) AS extract(second FROM to_timestamp(c) - j)#x] +Project [extract(second, cast(to_timestamp(c#x, None, TimestampType, Some(America/Los_Angeles), true) - j#x as timestamp)) AS extract(second FROM to_timestamp(c) - j)#x] +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation @@ -1432,7 +1432,7 @@ Project [datepart(year, c#x) AS datepart(year FROM c)#x, datepart(year, ntz#x) A +- SubqueryAlias t +- View (`t`, [c#x, ntz#x, i#x, j#x]) +- Project [cast(c#x as string) AS c#x, cast(ntz#x as timestamp_ntz) AS ntz#x, cast(i#x as interval year to month) AS i#x, cast(j#x as interval day to second) AS j#x] - +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), false) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] + +- Project [2011-05-06 07:08:09.1234567 AS c#x, to_timestamp_ntz(2011-05-06 07:08:09.1234567, None, TimestampNTZType, Some(America/Los_Angeles), true) AS ntz#x, INTERVAL '11-8' YEAR TO MONTH AS i#x, INTERVAL '31 16:50:06.789' DAY TO SECOND AS j#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out index da3f3de3fb448..b7dd089ba86a8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out @@ -60,7 +60,7 @@ SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FR GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2 -- !query analysis Sort [(i + 1)#x ASC NULLS FIRST, substring(s, 2, 3)#x ASC NULLS FIRST, extract(year FROM t)#x ASC NULLS FIRST, (d / 2)#x ASC NULLS FIRST, size(a)#x ASC NULLS FIRST, (f / i)#x ASC NULLS FIRST], true -+- Aggregate [(i#x + 1), (f#x / cast(i#x as decimal(10,0))), substring(s#x, 2, 3), extract(year, t#x), (cast(d#x as double) / cast(2 as double)), size(a#x, true)], [(i#x + 1) AS (i + 1)#x, (f#x / cast(i#x as decimal(10,0))) AS (f / i)#x, substring(s#x, 2, 3) AS substring(s, 2, 3)#x, extract(year, t#x) AS extract(year FROM t)#x, (cast(d#x as double) / cast(2 as double)) AS (d / 2)#x, size(a#x, true) AS size(a)#x] ++- Aggregate [(i#x + 1), (f#x / cast(i#x as decimal(10,0))), substring(s#x, 2, 3), extract(year, t#x), (cast(cast(d#x as bigint) as double) / cast(cast(2 as bigint) as double)), size(a#x, false)], [(i#x + 1) AS (i + 1)#x, (f#x / cast(i#x as decimal(10,0))) AS (f / i)#x, substring(s#x, 2, 3) AS substring(s, 2, 3)#x, extract(year, t#x) AS extract(year FROM t)#x, (cast(cast(d#x as bigint) as double) / cast(cast(2 as bigint) as double)) AS (d / 2)#x, size(a#x, false) AS size(a)#x] +- SubqueryAlias stuff +- View (`stuff`, [i#x, f#x, s#x, t#x, d#x, a#x]) +- Project [cast(i#x as int) AS i#x, cast(f#x as decimal(6,4)) AS f#x, cast(s#x as string) AS s#x, cast(t#x as string) AS t#x, cast(d#x as string) AS d#x, cast(a#x as array) AS a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out index 94d39111b29ed..dd36da7723556 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-filter.sql.out @@ -119,7 +119,7 @@ SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp -- !query SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp -- !query analysis -Aggregate [count(id#x) FILTER (WHERE (hiredate#x = to_date(2001-01-01 00:00:00, None, Some(America/Los_Angeles), false))) AS count(id) FILTER (WHERE (hiredate = to_date(2001-01-01 00:00:00)))#xL] +Aggregate [count(id#x) FILTER (WHERE (hiredate#x = to_date(2001-01-01 00:00:00, None, Some(America/Los_Angeles), true))) AS count(id) FILTER (WHERE (hiredate = to_date(2001-01-01 00:00:00)))#xL] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -131,7 +131,7 @@ Aggregate [count(id#x) FILTER (WHERE (hiredate#x = to_date(2001-01-01 00:00:00, -- !query SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp -- !query analysis -Aggregate [count(id#x) FILTER (WHERE (cast(hiredate#x as timestamp) = to_timestamp(2001-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), false))) AS count(id) FILTER (WHERE (hiredate = to_timestamp(2001-01-01 00:00:00)))#xL] +Aggregate [count(id#x) FILTER (WHERE (cast(hiredate#x as timestamp) = to_timestamp(2001-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), true))) AS count(id) FILTER (WHERE (hiredate = to_timestamp(2001-01-01 00:00:00)))#xL] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -179,7 +179,7 @@ Aggregate [count(distinct id#x) AS count(DISTINCT id)#xL, count(distinct id#x) F -- !query SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp -- !query analysis -Aggregate [count(distinct id#x) FILTER (WHERE (cast(hiredate#x as timestamp) = to_timestamp(2001-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), false))) AS count(DISTINCT id) FILTER (WHERE (hiredate = to_timestamp(2001-01-01 00:00:00)))#xL, count(distinct id#x) FILTER (WHERE (hiredate#x = to_date(2001-01-01 00:00:00, None, Some(America/Los_Angeles), false))) AS count(DISTINCT id) FILTER (WHERE (hiredate = to_date(2001-01-01 00:00:00)))#xL] +Aggregate [count(distinct id#x) FILTER (WHERE (cast(hiredate#x as timestamp) = to_timestamp(2001-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), true))) AS count(DISTINCT id) FILTER (WHERE (hiredate = to_timestamp(2001-01-01 00:00:00)))#xL, count(distinct id#x) FILTER (WHERE (hiredate#x = to_date(2001-01-01 00:00:00, None, Some(America/Los_Angeles), true))) AS count(DISTINCT id) FILTER (WHERE (hiredate = to_date(2001-01-01 00:00:00)))#xL] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -373,7 +373,7 @@ SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp -- !query SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id -- !query analysis -Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (hiredate#x > to_date(2003-01-01, None, Some(America/Los_Angeles), false))) AS sum(salary) FILTER (WHERE (hiredate > to_date(2003-01-01)))#x] +Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (hiredate#x > to_date(2003-01-01, None, Some(America/Los_Angeles), true))) AS sum(salary) FILTER (WHERE (hiredate > to_date(2003-01-01)))#x] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -385,7 +385,7 @@ Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (hiredate#x > to_ -- !query SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id -- !query analysis -Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (cast(hiredate#x as timestamp) > to_timestamp(2003-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), false))) AS sum(salary) FILTER (WHERE (hiredate > to_timestamp(2003-01-01 00:00:00)))#x] +Aggregate [dept_id#x], [dept_id#x, sum(salary#x) FILTER (WHERE (cast(hiredate#x as timestamp) > to_timestamp(2003-01-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), true))) AS sum(salary) FILTER (WHERE (hiredate > to_timestamp(2003-01-01 00:00:00)))#x] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -481,7 +481,7 @@ SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp -- !query SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_date("2003-01-01")) FROM emp GROUP BY 1 -- !query analysis -Aggregate [foo], [foo AS foo#x, sum(salary#x) FILTER (WHERE (hiredate#x >= to_date(2003-01-01, None, Some(America/Los_Angeles), false))) AS sum(salary) FILTER (WHERE (hiredate >= to_date(2003-01-01)))#x] +Aggregate [foo], [foo AS foo#x, sum(salary#x) FILTER (WHERE (hiredate#x >= to_date(2003-01-01, None, Some(America/Los_Angeles), true))) AS sum(salary) FILTER (WHERE (hiredate >= to_date(2003-01-01)))#x] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] @@ -493,7 +493,7 @@ Aggregate [foo], [foo AS foo#x, sum(salary#x) FILTER (WHERE (hiredate#x >= to_da -- !query SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_timestamp("2003-01-01")) FROM emp GROUP BY 1 -- !query analysis -Aggregate [foo], [foo AS foo#x, sum(salary#x) FILTER (WHERE (cast(hiredate#x as timestamp) >= to_timestamp(2003-01-01, None, TimestampType, Some(America/Los_Angeles), false))) AS sum(salary) FILTER (WHERE (hiredate >= to_timestamp(2003-01-01)))#x] +Aggregate [foo], [foo AS foo#x, sum(salary#x) FILTER (WHERE (cast(hiredate#x as timestamp) >= to_timestamp(2003-01-01, None, TimestampType, Some(America/Los_Angeles), true))) AS sum(salary) FILTER (WHERE (hiredate >= to_timestamp(2003-01-01)))#x] +- SubqueryAlias emp +- View (`EMP`, [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]) +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out index 8849aa4452252..34ff2a2186f0b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by.sql.out @@ -700,25 +700,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT every("true") -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"true\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"BOOLEAN\"", - "sqlExpr" : "\"every(true)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "every(\"true\")" - } ] -} +Aggregate [every(cast(true as boolean)) AS every(true)#x] ++- OneRowRelation -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out index 1281b19eb2f86..c06d1e5534aed 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/higher-order-functions.sql.out @@ -171,7 +171,7 @@ Project [aggregate(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct -- !query select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested -- !query analysis -Project [transform(zs#x, lambdafunction(aggregate(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, true)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +Project [transform(zs#x, lambdafunction(aggregate(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, false)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] @@ -211,7 +211,7 @@ Project [reduce(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(co -- !query select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested -- !query analysis -Project [transform(zs#x, lambdafunction(reduce(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, true)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +Project [transform(zs#x, lambdafunction(reduce(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, false)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] @@ -247,7 +247,7 @@ Project [exists(cast(null as array), lambdafunction((lambda y#x > 30), lamb -- !query select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested -- !query analysis -Project [zip_with(ys#x, zs#x, lambdafunction((lambda a#x + size(lambda b#x, true)), lambda a#x, lambda b#x, false)) AS v#x] +Project [zip_with(ys#x, zs#x, lambdafunction((lambda a#x + size(lambda b#x, false)), lambda a#x, lambda b#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out index c0196bbe118ef..c023e3b56f117 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/interval.sql.out @@ -605,63 +605,63 @@ Project [INTERVAL '59' SECOND AS INTERVAL '59' SECOND#x] -- !query select make_interval(1) -- !query analysis -Project [make_interval(1, 0, 0, 0, 0, 0, 0.000000, false) AS make_interval(1, 0, 0, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 0, 0, 0, 0, 0, 0.000000, true) AS make_interval(1, 0, 0, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2) -- !query analysis -Project [make_interval(1, 2, 0, 0, 0, 0, 0.000000, false) AS make_interval(1, 2, 0, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 0, 0, 0, 0, 0.000000, true) AS make_interval(1, 2, 0, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3) -- !query analysis -Project [make_interval(1, 2, 3, 0, 0, 0, 0.000000, false) AS make_interval(1, 2, 3, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 0, 0, 0, 0.000000, true) AS make_interval(1, 2, 3, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4) -- !query analysis -Project [make_interval(1, 2, 3, 4, 0, 0, 0.000000, false) AS make_interval(1, 2, 3, 4, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 0, 0, 0.000000, true) AS make_interval(1, 2, 3, 4, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 0, 0.000000, false) AS make_interval(1, 2, 3, 4, 5, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 5, 0, 0.000000, true) AS make_interval(1, 2, 3, 4, 5, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5, 6) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 6, 0.000000, false) AS make_interval(1, 2, 3, 4, 5, 6, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 5, 6, 0.000000, true) AS make_interval(1, 2, 3, 4, 5, 6, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 6, cast(7.008009 as decimal(18,6)), false) AS make_interval(1, 2, 3, 4, 5, 6, 7.008009)#x] +Project [make_interval(1, 2, 3, 4, 5, 6, cast(7.008009 as decimal(18,6)), true) AS make_interval(1, 2, 3, 4, 5, 6, 7.008009)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) -- !query analysis -Project [make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456, false) AS make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456)#x] +Project [make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456, true) AS make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456)#x] +- OneRowRelation -- !query select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) -- !query analysis -Project [make_interval(0, 0, 0, 0, 0, 0, cast(1234567890123456789 as decimal(18,6)), false) AS make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)#x] +Project [make_interval(0, 0, 0, 0, 0, 0, cast(1234567890123456789 as decimal(18,6)), true) AS make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out index fef9d0c5b6250..842b190c5a753 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/json-functions.sql.out @@ -9,7 +9,7 @@ Project [to_json(named_struct(a, 1, b, 2), Some(America/Los_Angeles)) AS to_json -- !query select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) -- !query analysis -Project [to_json((timestampFormat,dd/MM/yyyy), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false)), Some(America/Los_Angeles)) AS to_json(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +Project [to_json((timestampFormat,dd/MM/yyyy), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true)), Some(America/Los_Angeles)) AS to_json(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out index cd8f0e043b9ae..177f73608fba9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/map.sql.out @@ -2,7 +2,7 @@ -- !query select element_at(map(1, 'a', 2, 'b'), 5) -- !query analysis -Project [element_at(map(1, a, 2, b), 5, None, false) AS element_at(map(1, a, 2, b), 5)#x] +Project [element_at(map(1, a, 2, b), 5, None, true) AS element_at(map(1, a, 2, b), 5)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/math.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/math.sql.out index 5fe1b69352f57..1fa7b7513993d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/math.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/math.sql.out @@ -394,42 +394,42 @@ Project [bround(-9223372036854775808, -1) AS bround(-9223372036854775808, -1)#xL -- !query SELECT conv('100', 2, 10) -- !query analysis -Project [conv(100, 2, 10, false) AS conv(100, 2, 10)#x] +Project [conv(100, 2, 10, true) AS conv(100, 2, 10)#x] +- OneRowRelation -- !query SELECT conv(-10, 16, -10) -- !query analysis -Project [conv(cast(-10 as string), 16, -10, false) AS conv(-10, 16, -10)#x] +Project [conv(cast(-10 as string), 16, -10, true) AS conv(-10, 16, -10)#x] +- OneRowRelation -- !query SELECT conv('9223372036854775808', 10, 16) -- !query analysis -Project [conv(9223372036854775808, 10, 16, false) AS conv(9223372036854775808, 10, 16)#x] +Project [conv(9223372036854775808, 10, 16, true) AS conv(9223372036854775808, 10, 16)#x] +- OneRowRelation -- !query SELECT conv('92233720368547758070', 10, 16) -- !query analysis -Project [conv(92233720368547758070, 10, 16, false) AS conv(92233720368547758070, 10, 16)#x] +Project [conv(92233720368547758070, 10, 16, true) AS conv(92233720368547758070, 10, 16)#x] +- OneRowRelation -- !query SELECT conv('9223372036854775807', 36, 10) -- !query analysis -Project [conv(9223372036854775807, 36, 10, false) AS conv(9223372036854775807, 36, 10)#x] +Project [conv(9223372036854775807, 36, 10, true) AS conv(9223372036854775807, 36, 10)#x] +- OneRowRelation -- !query SELECT conv('-9223372036854775807', 36, 10) -- !query analysis -Project [conv(-9223372036854775807, 36, 10, false) AS conv(-9223372036854775807, 36, 10)#x] +Project [conv(-9223372036854775807, 36, 10, true) AS conv(-9223372036854775807, 36, 10)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/array.sql.out similarity index 93% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/array.sql.out index 53595d1b8a3eb..4db56d6c70561 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/array.sql.out @@ -212,7 +212,7 @@ select size(timestamp_array) from primitive_arrays -- !query analysis -Project [size(boolean_array#x, false) AS size(boolean_array)#x, size(tinyint_array#x, false) AS size(tinyint_array)#x, size(smallint_array#x, false) AS size(smallint_array)#x, size(int_array#x, false) AS size(int_array)#x, size(bigint_array#x, false) AS size(bigint_array)#x, size(decimal_array#x, false) AS size(decimal_array)#x, size(double_array#x, false) AS size(double_array)#x, size(float_array#x, false) AS size(float_array)#x, size(date_array#x, false) AS size(date_array)#x, size(timestamp_array#x, false) AS size(timestamp_array)#x] +Project [size(boolean_array#x, true) AS size(boolean_array)#x, size(tinyint_array#x, true) AS size(tinyint_array)#x, size(smallint_array#x, true) AS size(smallint_array)#x, size(int_array#x, true) AS size(int_array)#x, size(bigint_array#x, true) AS size(bigint_array)#x, size(decimal_array#x, true) AS size(decimal_array)#x, size(double_array#x, true) AS size(double_array)#x, size(float_array#x, true) AS size(float_array)#x, size(date_array#x, true) AS size(date_array)#x, size(timestamp_array#x, true) AS size(timestamp_array)#x] +- SubqueryAlias primitive_arrays +- View (`primitive_arrays`, [boolean_array#x, tinyint_array#x, smallint_array#x, int_array#x, bigint_array#x, decimal_array#x, double_array#x, float_array#x, date_array#x, timestamp_array#x]) +- Project [cast(boolean_array#x as array) AS boolean_array#x, cast(tinyint_array#x as array) AS tinyint_array#x, cast(smallint_array#x as array) AS smallint_array#x, cast(int_array#x as array) AS int_array#x, cast(bigint_array#x as array) AS bigint_array#x, cast(decimal_array#x as array) AS decimal_array#x, cast(double_array#x as array) AS double_array#x, cast(float_array#x as array) AS float_array#x, cast(date_array#x as array) AS date_array#x, cast(timestamp_array#x as array) AS timestamp_array#x] @@ -224,70 +224,70 @@ Project [size(boolean_array#x, false) AS size(boolean_array)#x, size(tinyint_arr -- !query select element_at(array(1, 2, 3), 5) -- !query analysis -Project [element_at(array(1, 2, 3), 5, None, true) AS element_at(array(1, 2, 3), 5)#x] +Project [element_at(array(1, 2, 3), 5, None, false) AS element_at(array(1, 2, 3), 5)#x] +- OneRowRelation -- !query select element_at(array(1, 2, 3), -5) -- !query analysis -Project [element_at(array(1, 2, 3), -5, None, true) AS element_at(array(1, 2, 3), -5)#x] +Project [element_at(array(1, 2, 3), -5, None, false) AS element_at(array(1, 2, 3), -5)#x] +- OneRowRelation -- !query select element_at(array(1, 2, 3), 0) -- !query analysis -Project [element_at(array(1, 2, 3), 0, None, true) AS element_at(array(1, 2, 3), 0)#x] +Project [element_at(array(1, 2, 3), 0, None, false) AS element_at(array(1, 2, 3), 0)#x] +- OneRowRelation -- !query select elt(4, '123', '456') -- !query analysis -Project [elt(4, 123, 456, true) AS elt(4, 123, 456)#x] +Project [elt(4, 123, 456, false) AS elt(4, 123, 456)#x] +- OneRowRelation -- !query select elt(0, '123', '456') -- !query analysis -Project [elt(0, 123, 456, true) AS elt(0, 123, 456)#x] +Project [elt(0, 123, 456, false) AS elt(0, 123, 456)#x] +- OneRowRelation -- !query select elt(-1, '123', '456') -- !query analysis -Project [elt(-1, 123, 456, true) AS elt(-1, 123, 456)#x] +Project [elt(-1, 123, 456, false) AS elt(-1, 123, 456)#x] +- OneRowRelation -- !query select elt(null, '123', '456') -- !query analysis -Project [elt(cast(null as int), 123, 456, true) AS elt(NULL, 123, 456)#x] +Project [elt(cast(null as int), 123, 456, false) AS elt(NULL, 123, 456)#x] +- OneRowRelation -- !query select elt(null, '123', null) -- !query analysis -Project [elt(cast(null as int), 123, cast(null as string), true) AS elt(NULL, 123, NULL)#x] +Project [elt(cast(null as int), 123, cast(null as string), false) AS elt(NULL, 123, NULL)#x] +- OneRowRelation -- !query select elt(1, '123', null) -- !query analysis -Project [elt(1, 123, cast(null as string), true) AS elt(1, 123, NULL)#x] +Project [elt(1, 123, cast(null as string), false) AS elt(1, 123, NULL)#x] +- OneRowRelation -- !query select elt(2, '123', null) -- !query analysis -Project [elt(2, 123, cast(null as string), true) AS elt(2, 123, NULL)#x] +Project [elt(2, 123, cast(null as string), false) AS elt(2, 123, NULL)#x] +- OneRowRelation @@ -360,21 +360,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2), false) AS size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2), true) AS size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +- OneRowRelation -- !query select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2, 3), false) AS size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10), 0, 1, 2, 3), true) AS size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))#x] +- OneRowRelation -- !query select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) -- !query analysis -Project [size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10), 0, 1, 2, 3), false) AS size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10)))#x] +Project [size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10), 0, 1, 2, 3), true) AS size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out similarity index 82% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out index 643dfd3771ffe..e0687b564d3d1 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/cast.sql.out @@ -205,193 +205,57 @@ Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] -- !query SELECT HEX(CAST(CAST(123 AS byte) AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 44, - "fragment" : "CAST(CAST(123 AS byte) AS binary)" - } ] -} +Project [hex(cast(cast(123 as tinyint) as binary)) AS hex(CAST(CAST(123 AS TINYINT) AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 45, - "fragment" : "CAST(CAST(-123 AS byte) AS binary)" - } ] -} +Project [hex(cast(cast(-123 as tinyint) as binary)) AS hex(CAST(CAST(-123 AS TINYINT) AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(123S AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"SMALLINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(123S AS binary)" - } ] -} +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(-123S AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"SMALLINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 32, - "fragment" : "CAST(-123S AS binary)" - } ] -} +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(123 AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"INT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 30, - "fragment" : "CAST(123 AS binary)" - } ] -} +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(-123 AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"INT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(-123 AS binary)" - } ] -} +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(123L AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"BIGINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(123L AS binary)" - } ] -} +Project [hex(cast(123 as binary)) AS hex(CAST(123 AS BINARY))#x] ++- OneRowRelation -- !query SELECT HEX(CAST(-123L AS binary)) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"BIGINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 32, - "fragment" : "CAST(-123L AS binary)" - } ] -} +Project [hex(cast(-123 as binary)) AS hex(CAST(-123 AS BINARY))#x] ++- OneRowRelation -- !query @@ -940,25 +804,8 @@ Project [hex(cast(abc as binary)) AS hex(CAST(abc AS BINARY))#x] -- !query SELECT HEX((123 :: byte) :: binary) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 34, - "fragment" : "(123 :: byte) :: binary" - } ] -} +Project [hex(cast(cast(123 as tinyint) as binary)) AS hex(CAST(CAST(123 AS TINYINT) AS BINARY))#x] ++- OneRowRelation -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/conditional-functions.sql.out similarity index 99% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/conditional-functions.sql.out index 5effa73c413a6..7df6556cddd4e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/conditional-functions.sql.out @@ -131,7 +131,7 @@ Project [zeroifnull(null) AS zeroifnull(NULL)#x, zeroifnull(1) AS zeroifnull(1)# -- !query SELECT zeroifnull('abc') -- !query analysis -Project [zeroifnull(abc) AS zeroifnull(abc)#xL] +Project [zeroifnull(abc) AS zeroifnull(abc)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/date.sql.out similarity index 84% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/date.sql.out index 0e4d2d4e99e26..88c7d7b4e7d72 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/date.sql.out @@ -37,21 +37,21 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query select make_date(2019, 1, 1), make_date(12, 12, 12) -- !query analysis -Project [make_date(2019, 1, 1, true) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, true) AS make_date(12, 12, 12)#x] +Project [make_date(2019, 1, 1, false) AS make_date(2019, 1, 1)#x, make_date(12, 12, 12, false) AS make_date(12, 12, 12)#x] +- OneRowRelation -- !query select make_date(2000, 13, 1) -- !query analysis -Project [make_date(2000, 13, 1, true) AS make_date(2000, 13, 1)#x] +Project [make_date(2000, 13, 1, false) AS make_date(2000, 13, 1)#x] +- OneRowRelation -- !query select make_date(2000, 1, 33) -- !query analysis -Project [make_date(2000, 1, 33, true) AS make_date(2000, 1, 33)#x] +Project [make_date(2000, 1, 33, false) AS make_date(2000, 1, 33)#x] +- OneRowRelation @@ -148,21 +148,21 @@ select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(n -- !query select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_date(cast(null as string), None, Some(America/Los_Angeles), true) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), true) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), true) AS to_date(2016-12-31, yyyy-MM-dd)#x] +Project [to_date(cast(null as string), None, Some(America/Los_Angeles), false) AS to_date(NULL)#x, to_date(2016-12-31, None, Some(America/Los_Angeles), false) AS to_date(2016-12-31)#x, to_date(2016-12-31, Some(yyyy-MM-dd), Some(America/Los_Angeles), false) AS to_date(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_date("16", "dd") -- !query analysis -Project [to_date(16, Some(dd), Some(America/Los_Angeles), true) AS to_date(16, dd)#x] +Project [to_date(16, Some(dd), Some(America/Los_Angeles), false) AS to_date(16, dd)#x] +- OneRowRelation -- !query select to_date("02-29", "MM-dd") -- !query analysis -Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), true) AS to_date(02-29, MM-dd)#x] +Project [to_date(02-29, Some(MM-dd), Some(America/Los_Angeles), false) AS to_date(02-29, MM-dd)#x] +- OneRowRelation @@ -201,21 +201,21 @@ select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(time -- !query select next_day("2015-07-23", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 as date), Mon, true) AS next_day(2015-07-23, Mon)#x] +Project [next_day(cast(2015-07-23 as date), Mon, false) AS next_day(2015-07-23, Mon)#x] +- OneRowRelation -- !query select next_day("2015-07-23", "xx") -- !query analysis -Project [next_day(cast(2015-07-23 as date), xx, true) AS next_day(2015-07-23, xx)#x] +Project [next_day(cast(2015-07-23 as date), xx, false) AS next_day(2015-07-23, xx)#x] +- OneRowRelation -- !query select next_day("2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(2015-07-23 12:12:12, Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(2015-07-23 12:12:12, Mon)#x] +- OneRowRelation @@ -228,28 +228,28 @@ select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") -- !query select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") -- !query analysis -Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, true) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +Project [next_day(cast(2015-07-23 12:12:12 as date), Mon, false) AS next_day(TIMESTAMP_NTZ '2015-07-23 12:12:12', Mon)#x] +- OneRowRelation -- !query select next_day("xx", "Mon") -- !query analysis -Project [next_day(cast(xx as date), Mon, true) AS next_day(xx, Mon)#x] +Project [next_day(cast(xx as date), Mon, false) AS next_day(xx, Mon)#x] +- OneRowRelation -- !query select next_day(null, "Mon") -- !query analysis -Project [next_day(cast(null as date), Mon, true) AS next_day(NULL, Mon)#x] +Project [next_day(cast(null as date), Mon, false) AS next_day(NULL, Mon)#x] +- OneRowRelation -- !query select next_day(null, "xx") -- !query analysis -Project [next_day(cast(null as date), xx, true) AS next_day(NULL, xx)#x] +Project [next_day(cast(null as date), xx, false) AS next_day(NULL, xx)#x] +- OneRowRelation @@ -355,15 +355,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select date_add('2011-11-11', '1') -- !query analysis -Project [date_add(cast(2011-11-11 as date), cast(1 as int)) AS date_add(2011-11-11, 1)#x] +Project [date_add(cast(2011-11-11 as date), 1) AS date_add(2011-11-11, 1)#x] +- OneRowRelation -- !query select date_add('2011-11-11', '1.2') -- !query analysis -Project [date_add(cast(2011-11-11 as date), cast(1.2 as int)) AS date_add(2011-11-11, 1.2)#x] -+- OneRowRelation +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_add" + } +} -- !query @@ -499,7 +505,14 @@ select date_sub(date'2011-11-11', '1') -- !query select date_sub(date'2011-11-11', '1.2') -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_sub" + } +} -- !query @@ -530,23 +543,49 @@ Project [date_sub(cast(2011-11-11 12:12:12 as date), 1) AS date_sub(TIMESTAMP_NT -- !query select date_add('2011-11-11', int_str) from date_view -- !query analysis -Project [date_add(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_add(2011-11-11, int_str)#x] -+- SubqueryAlias date_view - +- View (`date_view`, [date_str#x, int_str#x]) - +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] - +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_add('2011-11-11', int_str)" + } ] +} -- !query select date_sub('2011-11-11', int_str) from date_view -- !query analysis -Project [date_sub(cast(2011-11-11 as date), cast(int_str#x as int)) AS date_sub(2011-11-11, int_str)#x] -+- SubqueryAlias date_view - +- View (`date_view`, [date_str#x, int_str#x]) - +- Project [cast(date_str#x as string) AS date_str#x, cast(int_str#x as string) AS int_str#x] - +- Project [2011-11-11 AS date_str#x, 1 AS int_str#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_sub('2011-11-11', int_str)" + } ] +} -- !query @@ -622,7 +661,25 @@ select date '2001-10-01' - date '2001-09-28' -- !query select date '2001-10-01' - '2001-09-28' -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "date '2001-10-01' - '2001-09-28'" + } ] +} -- !query @@ -652,7 +709,25 @@ select date_str - date '2001-09-28' from date_view -- !query select date '2001-09-28' - date_str from date_view -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "date '2001-09-28' - date_str" + } ] +} -- !query @@ -664,7 +739,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DATE\"", + "inputType" : "\"DOUBLE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -687,11 +762,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"DATE '2011-11-11'\"", - "inputType" : "\"DATE\"", + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" }, "queryContext" : [ { "objectType" : "", @@ -729,7 +804,7 @@ select date '2012-01-01' - interval '2-2' year to month, -- !query select to_date('26/October/2015', 'dd/MMMMM/yyyy') -- !query analysis -Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), true) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +Project [to_date(26/October/2015, Some(dd/MMMMM/yyyy), Some(America/Los_Angeles), false) AS to_date(26/October/2015, dd/MMMMM/yyyy)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-parsing-invalid.sql.out similarity index 71% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-parsing-invalid.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-parsing-invalid.sql.out index 74146ab17a4d4..ad4a2feb9661c 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-parsing-invalid.sql.out @@ -2,126 +2,126 @@ -- !query select to_timestamp('294248', 'y') -- !query analysis -Project [to_timestamp(294248, Some(y), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(294248, y)#x] +Project [to_timestamp(294248, Some(y), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(294248, y)#x] +- OneRowRelation -- !query select to_timestamp('1', 'yy') -- !query analysis -Project [to_timestamp(1, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, yy)#x] +Project [to_timestamp(1, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, yy)#x] +- OneRowRelation -- !query select to_timestamp('-12', 'yy') -- !query analysis -Project [to_timestamp(-12, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(-12, yy)#x] +Project [to_timestamp(-12, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(-12, yy)#x] +- OneRowRelation -- !query select to_timestamp('123', 'yy') -- !query analysis -Project [to_timestamp(123, Some(yy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(123, yy)#x] +Project [to_timestamp(123, Some(yy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123, yy)#x] +- OneRowRelation -- !query select to_timestamp('1', 'yyy') -- !query analysis -Project [to_timestamp(1, Some(yyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1, yyy)#x] +Project [to_timestamp(1, Some(yyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1, yyy)#x] +- OneRowRelation -- !query select to_timestamp('1234567', 'yyyyyyy') -- !query analysis -Project [to_timestamp(1234567, Some(yyyyyyy), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1234567, yyyyyyy)#x] +Project [to_timestamp(1234567, Some(yyyyyyy), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1234567, yyyyyyy)#x] +- OneRowRelation -- !query select to_timestamp('366', 'D') -- !query analysis -Project [to_timestamp(366, Some(D), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(366, D)#x] +Project [to_timestamp(366, Some(D), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(366, D)#x] +- OneRowRelation -- !query select to_timestamp('9', 'DD') -- !query analysis -Project [to_timestamp(9, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, DD)#x] +Project [to_timestamp(9, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, DD)#x] +- OneRowRelation -- !query select to_timestamp('366', 'DD') -- !query analysis -Project [to_timestamp(366, Some(DD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(366, DD)#x] +Project [to_timestamp(366, Some(DD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(366, DD)#x] +- OneRowRelation -- !query select to_timestamp('9', 'DDD') -- !query analysis -Project [to_timestamp(9, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(9, DDD)#x] +Project [to_timestamp(9, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(9, DDD)#x] +- OneRowRelation -- !query select to_timestamp('99', 'DDD') -- !query analysis -Project [to_timestamp(99, Some(DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(99, DDD)#x] +Project [to_timestamp(99, Some(DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(99, DDD)#x] +- OneRowRelation -- !query select to_timestamp('30-365', 'dd-DDD') -- !query analysis -Project [to_timestamp(30-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(30-365, dd-DDD)#x] +Project [to_timestamp(30-365, Some(dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(30-365, dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('11-365', 'MM-DDD') -- !query analysis -Project [to_timestamp(11-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(11-365, MM-DDD)#x] +Project [to_timestamp(11-365, Some(MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(11-365, MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2019-366', 'yyyy-DDD') -- !query analysis -Project [to_timestamp(2019-366, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-366, yyyy-DDD)#x] +Project [to_timestamp(2019-366, Some(yyyy-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-366, yyyy-DDD)#x] +- OneRowRelation -- !query select to_timestamp('12-30-365', 'MM-dd-DDD') -- !query analysis -Project [to_timestamp(12-30-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12-30-365, MM-dd-DDD)#x] +Project [to_timestamp(12-30-365, Some(MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12-30-365, MM-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-01-365', 'yyyy-dd-DDD') -- !query analysis -Project [to_timestamp(2020-01-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-01-365, yyyy-dd-DDD)#x] +Project [to_timestamp(2020-01-365, Some(yyyy-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-01-365, yyyy-dd-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-10-350', 'yyyy-MM-DDD') -- !query analysis -Project [to_timestamp(2020-10-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-10-350, yyyy-MM-DDD)#x] +Project [to_timestamp(2020-10-350, Some(yyyy-MM-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-10-350, yyyy-MM-DDD)#x] +- OneRowRelation -- !query select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') -- !query analysis -Project [to_timestamp(2020-11-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-11-31-366, yyyy-MM-dd-DDD)#x] +Project [to_timestamp(2020-11-31-366, Some(yyyy-MM-dd-DDD), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-11-31-366, yyyy-MM-dd-DDD)#x] +- OneRowRelation @@ -135,56 +135,56 @@ Project [from_csv(StructField(date,DateType,true), (dateFormat,yyyy-DDD), 2018-3 -- !query select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_date(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), true) AS to_date(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_date(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), false) AS to_date(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_date(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), true) AS to_date(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_date(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), Some(America/Los_Angeles), false) AS to_date(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_timestamp(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_timestamp(2020-01-27T20:06:11.847, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_timestamp(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +Project [to_timestamp(Unparseable, Some(yyyy-MM-dd HH:mm:ss.SSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#x] +- OneRowRelation -- !query select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS to_unix_timestamp(2020-01-27T20:06:11.847, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation -- !query select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query analysis -Project [to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), true) AS to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +Project [to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS, Some(America/Los_Angeles), false) AS to_unix_timestamp(Unparseable, yyyy-MM-dd HH:mm:ss.SSS)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-special.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-special.sql.out similarity index 56% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-special.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-special.sql.out index 6768297fd8116..01d1f2c40a4a6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/datetime-special.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/datetime-special.sql.out @@ -8,7 +8,7 @@ select date'999999-03-18', date'-0001-1-28', date'0015' -- !query select make_date(999999, 3, 18), make_date(-1, 1, 28) -- !query analysis -Project [make_date(999999, 3, 18, true) AS make_date(999999, 3, 18)#x, make_date(-1, 1, 28, true) AS make_date(-1, 1, 28)#x] +Project [make_date(999999, 3, 18, false) AS make_date(999999, 3, 18)#x, make_date(-1, 1, 28, false) AS make_date(-1, 1, 28)#x] +- OneRowRelation @@ -21,5 +21,5 @@ select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timesta -- !query select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) -- !query analysis -Project [make_timestamp(-1969, 12, 31, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(-1969, 12, 31, 16, 0, 0.0)#x, make_timestamp(-15, 3, 18, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(-15, 3, 18, 16, 0, 0.0)#x, make_timestamp(99999, 3, 18, 12, 3, cast(17.0 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(99999, 3, 18, 12, 3, 17.0)#x] +Project [make_timestamp(-1969, 12, 31, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(-1969, 12, 31, 16, 0, 0.0)#x, make_timestamp(-15, 3, 18, 16, 0, cast(0.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(-15, 3, 18, 16, 0, 0.0)#x, make_timestamp(99999, 3, 18, 12, 3, cast(17.0 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(99999, 3, 18, 12, 3, 17.0)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-disabled.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-disabled.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out similarity index 56% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out index 22dfeac5fd0b6..a02bf525f947d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out @@ -2,32 +2,27 @@ -- !query SELECT 1 FROM "not_exist" -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 25, - "fragment" : "\"not_exist\"" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } -- !query USE SCHEMA "not_exist" -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "SCHEMA_NOT_FOUND", - "sqlState" : "42704", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "schemaName" : "`spark_catalog`.`not_exist`" + "error" : "'\"not_exist\"'", + "hint" : "" } } @@ -35,122 +30,84 @@ org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException -- !query ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 23, - "fragment" : "\"not_exist\"" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } -- !query ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "not_exist" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } -- !query SELECT 1 AS "not_exist" FROM not_exist -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 30, - "stopIndex" : 38, - "fragment" : "not_exist" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } -- !query SELECT 1 FROM not_exist AS X("hello") -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 23, - "fragment" : "not_exist" - } ] + "error" : "'\"hello\"'", + "hint" : "" + } } -- !query SELECT "not_exist"() -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "UNRESOLVED_ROUTINE", - "sqlState" : "42883", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "routineName" : "`not_exist`", - "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "\"not_exist\"()" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } -- !query SELECT "not_exist".not_exist() -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "UNRESOLVED_ROUTINE", - "sqlState" : "42883", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "routineName" : "`not_exist`.`not_exist`", - "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "\"not_exist\".not_exist()" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -312,62 +269,29 @@ org.apache.spark.sql.AnalysisException -- !query SELECT "hello" -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`hello`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 14, - "fragment" : "\"hello\"" - } ] -} +Project [hello AS hello#x] ++- OneRowRelation -- !query CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"hello\"'", - "hint" : "" - } -} +CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, UNSUPPORTED, true + +- Project [1 AS 1#x] + +- OneRowRelation -- !query DROP VIEW v -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`v`" - } -} +DropTempViewCommand v -- !query SELECT INTERVAL "1" YEAR -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"1\"'", - "hint" : "" - } -} +Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] ++- OneRowRelation -- !query @@ -401,54 +325,69 @@ Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] -- !query CREATE SCHEMA "myschema" -- !query analysis -CreateNamespace false -+- ResolvedNamespace V2SessionCatalog(spark_catalog), [myschema] +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} -- !query CREATE TEMPORARY VIEW "myview"("c1") AS WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v" -- !query analysis -CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, UNSUPPORTED, true - +- WithCTE - :- CTERelationDef xxxx, false - : +- SubqueryAlias v - : +- Project [1#x AS a#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [a#x] - +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} -- !query SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") -- !query analysis -Project [a1#x AS a2#x] -+- SubqueryAlias atab - +- Project [c1#x AS a1#x] - +- SubqueryAlias myview - +- View (`myview`, [c1#x]) - +- Project [cast(a#x as int) AS c1#x] - +- WithCTE - :- CTERelationDef xxxx, false - : +- SubqueryAlias v - : +- Project [1#x AS a#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [a#x] - +- SubqueryAlias v - +- CTERelationRef xxxx, true, [a#x], false +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} -- !query DROP TABLE "myview" -- !query analysis -DropTempViewCommand myview +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} -- !query DROP SCHEMA "myschema" -- !query analysis -DropNamespace false, false -+- ResolvedNamespace V2SessionCatalog(spark_catalog), [myschema] +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out similarity index 97% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/higher-order-functions.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out index c06d1e5534aed..1281b19eb2f86 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/higher-order-functions.sql.out @@ -171,7 +171,7 @@ Project [aggregate(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct -- !query select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested -- !query analysis -Project [transform(zs#x, lambdafunction(aggregate(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, false)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +Project [transform(zs#x, lambdafunction(aggregate(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, true)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] @@ -211,7 +211,7 @@ Project [reduce(ys#x, named_struct(sum, 0, n, 0), lambdafunction(named_struct(co -- !query select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested -- !query analysis -Project [transform(zs#x, lambdafunction(reduce(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, false)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +Project [transform(zs#x, lambdafunction(reduce(lambda z#x, 1, lambdafunction(((lambda acc#x * lambda val#x) * size(lambda z#x, true)), lambda acc#x, lambda val#x, false), lambdafunction(lambda id#x, lambda id#x, false)), lambda z#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] @@ -247,7 +247,7 @@ Project [exists(cast(null as array), lambdafunction((lambda y#x > 30), lamb -- !query select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested -- !query analysis -Project [zip_with(ys#x, zs#x, lambdafunction((lambda a#x + size(lambda b#x, false)), lambda a#x, lambda b#x, false)) AS v#x] +Project [zip_with(ys#x, zs#x, lambdafunction((lambda a#x + size(lambda b#x, true)), lambda a#x, lambda b#x, false)) AS v#x] +- SubqueryAlias nested +- View (`nested`, [x#x, ys#x, zs#x]) +- Project [cast(x#x as int) AS x#x, cast(ys#x as array) AS ys#x, cast(zs#x as array>) AS zs#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out similarity index 98% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out index c023e3b56f117..c0196bbe118ef 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/interval.sql.out @@ -605,63 +605,63 @@ Project [INTERVAL '59' SECOND AS INTERVAL '59' SECOND#x] -- !query select make_interval(1) -- !query analysis -Project [make_interval(1, 0, 0, 0, 0, 0, 0.000000, true) AS make_interval(1, 0, 0, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 0, 0, 0, 0, 0, 0.000000, false) AS make_interval(1, 0, 0, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2) -- !query analysis -Project [make_interval(1, 2, 0, 0, 0, 0, 0.000000, true) AS make_interval(1, 2, 0, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 0, 0, 0, 0, 0.000000, false) AS make_interval(1, 2, 0, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3) -- !query analysis -Project [make_interval(1, 2, 3, 0, 0, 0, 0.000000, true) AS make_interval(1, 2, 3, 0, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 0, 0, 0, 0.000000, false) AS make_interval(1, 2, 3, 0, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4) -- !query analysis -Project [make_interval(1, 2, 3, 4, 0, 0, 0.000000, true) AS make_interval(1, 2, 3, 4, 0, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 0, 0, 0.000000, false) AS make_interval(1, 2, 3, 4, 0, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 0, 0.000000, true) AS make_interval(1, 2, 3, 4, 5, 0, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 5, 0, 0.000000, false) AS make_interval(1, 2, 3, 4, 5, 0, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5, 6) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 6, 0.000000, true) AS make_interval(1, 2, 3, 4, 5, 6, 0.000000)#x] +Project [make_interval(1, 2, 3, 4, 5, 6, 0.000000, false) AS make_interval(1, 2, 3, 4, 5, 6, 0.000000)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 5, 6, 7.008009) -- !query analysis -Project [make_interval(1, 2, 3, 4, 5, 6, cast(7.008009 as decimal(18,6)), true) AS make_interval(1, 2, 3, 4, 5, 6, 7.008009)#x] +Project [make_interval(1, 2, 3, 4, 5, 6, cast(7.008009 as decimal(18,6)), false) AS make_interval(1, 2, 3, 4, 5, 6, 7.008009)#x] +- OneRowRelation -- !query select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) -- !query analysis -Project [make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456, true) AS make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456)#x] +Project [make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456, false) AS make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456)#x] +- OneRowRelation -- !query select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) -- !query analysis -Project [make_interval(0, 0, 0, 0, 0, 0, cast(1234567890123456789 as decimal(18,6)), true) AS make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)#x] +Project [make_interval(0, 0, 0, 0, 0, 0, cast(1234567890123456789 as decimal(18,6)), false) AS make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/keywords.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/keywords.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/keywords.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/literals.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/map.sql.out similarity index 97% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/map.sql.out index 177f73608fba9..cd8f0e043b9ae 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/map.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/map.sql.out @@ -2,7 +2,7 @@ -- !query select element_at(map(1, 'a', 2, 'b'), 5) -- !query analysis -Project [element_at(map(1, a, 2, b), 5, None, true) AS element_at(map(1, a, 2, b), 5)#x] +Project [element_at(map(1, a, 2, b), 5, None, false) AS element_at(map(1, a, 2, b), 5)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/math.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/math.sql.out similarity index 94% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/math.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/math.sql.out index 1fa7b7513993d..5fe1b69352f57 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/math.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/math.sql.out @@ -394,42 +394,42 @@ Project [bround(-9223372036854775808, -1) AS bround(-9223372036854775808, -1)#xL -- !query SELECT conv('100', 2, 10) -- !query analysis -Project [conv(100, 2, 10, true) AS conv(100, 2, 10)#x] +Project [conv(100, 2, 10, false) AS conv(100, 2, 10)#x] +- OneRowRelation -- !query SELECT conv(-10, 16, -10) -- !query analysis -Project [conv(cast(-10 as string), 16, -10, true) AS conv(-10, 16, -10)#x] +Project [conv(cast(-10 as string), 16, -10, false) AS conv(-10, 16, -10)#x] +- OneRowRelation -- !query SELECT conv('9223372036854775808', 10, 16) -- !query analysis -Project [conv(9223372036854775808, 10, 16, true) AS conv(9223372036854775808, 10, 16)#x] +Project [conv(9223372036854775808, 10, 16, false) AS conv(9223372036854775808, 10, 16)#x] +- OneRowRelation -- !query SELECT conv('92233720368547758070', 10, 16) -- !query analysis -Project [conv(92233720368547758070, 10, 16, true) AS conv(92233720368547758070, 10, 16)#x] +Project [conv(92233720368547758070, 10, 16, false) AS conv(92233720368547758070, 10, 16)#x] +- OneRowRelation -- !query SELECT conv('9223372036854775807', 36, 10) -- !query analysis -Project [conv(9223372036854775807, 36, 10, true) AS conv(9223372036854775807, 36, 10)#x] +Project [conv(9223372036854775807, 36, 10, false) AS conv(9223372036854775807, 36, 10)#x] +- OneRowRelation -- !query SELECT conv('-9223372036854775807', 36, 10) -- !query analysis -Project [conv(-9223372036854775807, 36, 10, true) AS conv(-9223372036854775807, 36, 10)#x] +Project [conv(-9223372036854775807, 36, 10, false) AS conv(-9223372036854775807, 36, 10)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/parse-schema-string.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/parse-schema-string.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/parse-schema-string.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/parse-schema-string.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/string-functions.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/string-functions.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out similarity index 76% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out index 560974d28c545..dcfd783b648f8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/timestamp.sql.out @@ -90,70 +90,70 @@ Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(Americ -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 1) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 60) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, null) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +- OneRowRelation @@ -302,231 +302,231 @@ select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020- -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp(1) -- !query analysis -Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1)#x] +Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1)#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") -- !query analysis -Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +- OneRowRelation -- !query select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") -- !query analysis -Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +- OneRowRelation -- !query select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") -- !query analysis -Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +- OneRowRelation -- !query select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("16", "dd") -- !query analysis -Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(16, dd)#x] +Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(16, dd)#x] +- OneRowRelation -- !query select to_timestamp("02-29", "MM-dd") -- !query analysis -Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(02-29, MM-dd)#x] +Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(02-29, MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019 40", "yyyy mm") -- !query analysis -Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 40, yyyy mm)#x] +Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 40, yyyy mm)#x] +- OneRowRelation -- !query select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") -- !query analysis -Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +- OneRowRelation @@ -551,13 +551,49 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" + } ] +} -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" + } ] +} -- !query @@ -583,13 +619,49 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "str - timestamp'2011-11-11 11:11:11'" + } ] +} -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -[Analyzer test output redacted due to nondeterminism] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} -- !query @@ -597,11 +669,11 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "actualDataType" : "\"TIMESTAMP\"", - "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -619,11 +691,11 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "actualDataType" : "\"TIMESTAMP\"", - "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { @@ -702,28 +774,28 @@ select date '2012-01-01' - interval 3 hours, -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query analysis -Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +- OneRowRelation -- !query select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), true) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_aggregates.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_aggregates.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_aggregates.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_aggregates.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_arithmetic.sql.out similarity index 99% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_arithmetic.sql.out index 30654d1d71e2b..caf997f6ccbb2 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_arithmetic.sql.out @@ -23,7 +23,7 @@ Project [try_add(2147483647, cast(1 as decimal(10,0))) AS try_add(2147483647, 1) -- !query SELECT try_add(2147483647, "1") -- !query analysis -Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#xL] +Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#x] +- OneRowRelation @@ -305,7 +305,7 @@ Project [try_subtract(2147483647, cast(-1 as decimal(10,0))) AS try_subtract(214 -- !query SELECT try_subtract(2147483647, "-1") -- !query analysis -Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#xL] +Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#x] +- OneRowRelation @@ -403,7 +403,7 @@ Project [try_multiply(2147483647, cast(-2 as decimal(10,0))) AS try_multiply(214 -- !query SELECT try_multiply(2147483647, "-2") -- !query analysis -Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#xL] +Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_datetime_functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_datetime_functions.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_datetime_functions.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_datetime_functions.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_element_at.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_element_at.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/ansi/try_element_at.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/try_element_at.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 47eb8f2417381..2e38ed137b415 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -1930,14 +1930,17 @@ Union false, false -- !query values (0, 1) tab(x, y) |> union table t +|> where x = 0 -- !query analysis Distinct +- Union false, false - :- Project [x#x, cast(y#x as string) AS y#x] + :- Project [x#x, cast(y#x as bigint) AS y#xL] : +- SubqueryAlias tab : +- LocalRelation [x#x, y#x] - +- SubqueryAlias spark_catalog.default.t - +- Relation spark_catalog.default.t[x#x,y#x] csv + +- Project [x#x, cast(y#x as bigint) AS y#xL] + +- Filter (x#x = 0) + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/predicate-functions.sql.out index 7e720995c44b4..55822a10041f5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/predicate-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/predicate-functions.sql.out @@ -156,7 +156,7 @@ Project [(1 = 1) AS (1 = 1)#x] -- !query select 1 = '1' -- !query analysis -Project [(1 = cast(1 as int)) AS (1 = 1)#x] +Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = 1)#x] +- OneRowRelation @@ -177,28 +177,28 @@ Project [(cast(1.5 as double) = cast(1.51 as double)) AS (1.5 = 1.51)#x] -- !query select 1 > '1' -- !query analysis -Project [(1 > cast(1 as int)) AS (1 > 1)#x] +Project [(cast(1 as bigint) > cast(1 as bigint)) AS (1 > 1)#x] +- OneRowRelation -- !query select 2 > '1.0' -- !query analysis -Project [(2 > cast(1.0 as int)) AS (2 > 1.0)#x] +Project [(cast(2 as bigint) > cast(1.0 as bigint)) AS (2 > 1.0)#x] +- OneRowRelation -- !query select 2 > '2.0' -- !query analysis -Project [(2 > cast(2.0 as int)) AS (2 > 2.0)#x] +Project [(cast(2 as bigint) > cast(2.0 as bigint)) AS (2 > 2.0)#x] +- OneRowRelation -- !query select 2 > '2.2' -- !query analysis -Project [(2 > cast(2.2 as int)) AS (2 > 2.2)#x] +Project [(cast(2 as bigint) > cast(2.2 as bigint)) AS (2 > 2.2)#x] +- OneRowRelation @@ -212,35 +212,35 @@ Project [(cast(1.5 as double) > cast(0.5 as double)) AS (1.5 > 0.5)#x] -- !query select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52') -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) > to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false)) AS (to_date(2009-07-30 04:17:52) > to_date(2009-07-30 04:17:52))#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) > to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true)) AS (to_date(2009-07-30 04:17:52) > to_date(2009-07-30 04:17:52))#x] +- OneRowRelation -- !query select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52' -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) > cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) > 2009-07-30 04:17:52)#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) > cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) > 2009-07-30 04:17:52)#x] +- OneRowRelation -- !query select 1 >= '1' -- !query analysis -Project [(1 >= cast(1 as int)) AS (1 >= 1)#x] +Project [(cast(1 as bigint) >= cast(1 as bigint)) AS (1 >= 1)#x] +- OneRowRelation -- !query select 2 >= '1.0' -- !query analysis -Project [(2 >= cast(1.0 as int)) AS (2 >= 1.0)#x] +Project [(cast(2 as bigint) >= cast(1.0 as bigint)) AS (2 >= 1.0)#x] +- OneRowRelation -- !query select 2 >= '2.0' -- !query analysis -Project [(2 >= cast(2.0 as int)) AS (2 >= 2.0)#x] +Project [(cast(2 as bigint) >= cast(2.0 as bigint)) AS (2 >= 2.0)#x] +- OneRowRelation @@ -261,35 +261,35 @@ Project [(cast(1.5 as double) >= cast(0.5 as double)) AS (1.5 >= 0.5)#x] -- !query select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52') -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) >= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false)) AS (to_date(2009-07-30 04:17:52) >= to_date(2009-07-30 04:17:52))#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) >= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true)) AS (to_date(2009-07-30 04:17:52) >= to_date(2009-07-30 04:17:52))#x] +- OneRowRelation -- !query select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52' -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) >= cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) >= 2009-07-30 04:17:52)#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) >= cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) >= 2009-07-30 04:17:52)#x] +- OneRowRelation -- !query select 1 < '1' -- !query analysis -Project [(1 < cast(1 as int)) AS (1 < 1)#x] +Project [(cast(1 as bigint) < cast(1 as bigint)) AS (1 < 1)#x] +- OneRowRelation -- !query select 2 < '1.0' -- !query analysis -Project [(2 < cast(1.0 as int)) AS (2 < 1.0)#x] +Project [(cast(2 as bigint) < cast(1.0 as bigint)) AS (2 < 1.0)#x] +- OneRowRelation -- !query select 2 < '2.0' -- !query analysis -Project [(2 < cast(2.0 as int)) AS (2 < 2.0)#x] +Project [(cast(2 as bigint) < cast(2.0 as bigint)) AS (2 < 2.0)#x] +- OneRowRelation @@ -310,35 +310,35 @@ Project [(cast(0.5 as double) < cast(1.5 as double)) AS (0.5 < 1.5)#x] -- !query select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52') -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) < to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false)) AS (to_date(2009-07-30 04:17:52) < to_date(2009-07-30 04:17:52))#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) < to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true)) AS (to_date(2009-07-30 04:17:52) < to_date(2009-07-30 04:17:52))#x] +- OneRowRelation -- !query select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52' -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) < cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) < 2009-07-30 04:17:52)#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) < cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) < 2009-07-30 04:17:52)#x] +- OneRowRelation -- !query select 1 <= '1' -- !query analysis -Project [(1 <= cast(1 as int)) AS (1 <= 1)#x] +Project [(cast(1 as bigint) <= cast(1 as bigint)) AS (1 <= 1)#x] +- OneRowRelation -- !query select 2 <= '1.0' -- !query analysis -Project [(2 <= cast(1.0 as int)) AS (2 <= 1.0)#x] +Project [(cast(2 as bigint) <= cast(1.0 as bigint)) AS (2 <= 1.0)#x] +- OneRowRelation -- !query select 2 <= '2.0' -- !query analysis -Project [(2 <= cast(2.0 as int)) AS (2 <= 2.0)#x] +Project [(cast(2 as bigint) <= cast(2.0 as bigint)) AS (2 <= 2.0)#x] +- OneRowRelation @@ -359,49 +359,49 @@ Project [(cast(0.5 as double) <= cast(1.5 as double)) AS (0.5 <= 1.5)#x] -- !query select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52') -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) <= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false)) AS (to_date(2009-07-30 04:17:52) <= to_date(2009-07-30 04:17:52))#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) <= to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true)) AS (to_date(2009-07-30 04:17:52) <= to_date(2009-07-30 04:17:52))#x] +- OneRowRelation -- !query select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52' -- !query analysis -Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), false) <= cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) <= 2009-07-30 04:17:52)#x] +Project [(to_date(2009-07-30 04:17:52, None, Some(America/Los_Angeles), true) <= cast(2009-07-30 04:17:52 as date)) AS (to_date(2009-07-30 04:17:52) <= 2009-07-30 04:17:52)#x] +- OneRowRelation -- !query select to_date('2017-03-01') = to_timestamp('2017-03-01 00:00:00') -- !query analysis -Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), false) as timestamp) = to_timestamp(2017-03-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), false)) AS (to_date(2017-03-01) = to_timestamp(2017-03-01 00:00:00))#x] +Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), true) as timestamp) = to_timestamp(2017-03-01 00:00:00, None, TimestampType, Some(America/Los_Angeles), true)) AS (to_date(2017-03-01) = to_timestamp(2017-03-01 00:00:00))#x] +- OneRowRelation -- !query select to_timestamp('2017-03-01 00:00:01') > to_date('2017-03-01') -- !query analysis -Project [(to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), false) > cast(to_date(2017-03-01, None, Some(America/Los_Angeles), false) as timestamp)) AS (to_timestamp(2017-03-01 00:00:01) > to_date(2017-03-01))#x] +Project [(to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), true) > cast(to_date(2017-03-01, None, Some(America/Los_Angeles), true) as timestamp)) AS (to_timestamp(2017-03-01 00:00:01) > to_date(2017-03-01))#x] +- OneRowRelation -- !query select to_timestamp('2017-03-01 00:00:01') >= to_date('2017-03-01') -- !query analysis -Project [(to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), false) >= cast(to_date(2017-03-01, None, Some(America/Los_Angeles), false) as timestamp)) AS (to_timestamp(2017-03-01 00:00:01) >= to_date(2017-03-01))#x] +Project [(to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), true) >= cast(to_date(2017-03-01, None, Some(America/Los_Angeles), true) as timestamp)) AS (to_timestamp(2017-03-01 00:00:01) >= to_date(2017-03-01))#x] +- OneRowRelation -- !query select to_date('2017-03-01') < to_timestamp('2017-03-01 00:00:01') -- !query analysis -Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), false) as timestamp) < to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), false)) AS (to_date(2017-03-01) < to_timestamp(2017-03-01 00:00:01))#x] +Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), true) as timestamp) < to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), true)) AS (to_date(2017-03-01) < to_timestamp(2017-03-01 00:00:01))#x] +- OneRowRelation -- !query select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01') -- !query analysis -Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), false) as timestamp) <= to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), false)) AS (to_date(2017-03-01) <= to_timestamp(2017-03-01 00:00:01))#x] +Project [(cast(to_date(2017-03-01, None, Some(America/Los_Angeles), true) as timestamp) <= to_timestamp(2017-03-01 00:00:01, None, TimestampType, Some(America/Los_Angeles), true)) AS (to_date(2017-03-01) <= to_timestamp(2017-03-01 00:00:01))#x] +- OneRowRelation @@ -436,14 +436,14 @@ Project [cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as dec -- !query select 1 in ('2', '3', '4') -- !query analysis -Project [cast(1 as string) IN (cast(2 as string),cast(3 as string),cast(4 as string)) AS (1 IN (2, 3, 4))#x] +Project [cast(1 as bigint) IN (cast(2 as bigint),cast(3 as bigint),cast(4 as bigint)) AS (1 IN (2, 3, 4))#x] +- OneRowRelation -- !query select 1 in ('2', '3', '4', null) -- !query analysis -Project [cast(1 as string) IN (cast(2 as string),cast(3 as string),cast(4 as string),cast(null as string)) AS (1 IN (2, 3, 4, NULL))#x] +Project [cast(1 as bigint) IN (cast(2 as bigint),cast(3 as bigint),cast(4 as bigint),cast(null as bigint)) AS (1 IN (2, 3, 4, NULL))#x] +- OneRowRelation @@ -492,14 +492,14 @@ Project [NOT cast(1 as decimal(11,1)) IN (cast(1.0 as decimal(11,1)),cast(2.0 as -- !query select 1 not in ('2', '3', '4') -- !query analysis -Project [NOT cast(1 as string) IN (cast(2 as string),cast(3 as string),cast(4 as string)) AS (NOT (1 IN (2, 3, 4)))#x] +Project [NOT cast(1 as bigint) IN (cast(2 as bigint),cast(3 as bigint),cast(4 as bigint)) AS (NOT (1 IN (2, 3, 4)))#x] +- OneRowRelation -- !query select 1 not in ('2', '3', '4', null) -- !query analysis -Project [NOT cast(1 as string) IN (cast(2 as string),cast(3 as string),cast(4 as string),cast(null as string)) AS (NOT (1 IN (2, 3, 4, NULL)))#x] +Project [NOT cast(1 as bigint) IN (cast(2 as bigint),cast(3 as bigint),cast(4 as bigint),cast(null as bigint)) AS (NOT (1 IN (2, 3, 4, NULL)))#x] +- OneRowRelation @@ -548,7 +548,7 @@ Project [between(b, a, c) AS between(b, a, c)#x] -- !query select to_timestamp('2022-12-26 00:00:01') between to_date('2022-03-01') and to_date('2022-12-31') -- !query analysis -Project [between(to_timestamp(2022-12-26 00:00:01, None, TimestampType, Some(America/Los_Angeles), false), to_date(2022-03-01, None, Some(America/Los_Angeles), false), to_date(2022-12-31, None, Some(America/Los_Angeles), false)) AS between(to_timestamp(2022-12-26 00:00:01), to_date(2022-03-01), to_date(2022-12-31))#x] +Project [between(to_timestamp(2022-12-26 00:00:01, None, TimestampType, Some(America/Los_Angeles), true), to_date(2022-03-01, None, Some(America/Los_Angeles), true), to_date(2022-12-31, None, Some(America/Los_Angeles), true)) AS between(to_timestamp(2022-12-26 00:00:01), to_date(2022-03-01), to_date(2022-12-31))#x] +- OneRowRelation @@ -589,7 +589,7 @@ Project [NOT between(b, a, c) AS (NOT between(b, a, c))#x] -- !query select to_timestamp('2022-12-26 00:00:01') not between to_date('2022-03-01') and to_date('2022-12-31') -- !query analysis -Project [NOT between(to_timestamp(2022-12-26 00:00:01, None, TimestampType, Some(America/Los_Angeles), false), to_date(2022-03-01, None, Some(America/Los_Angeles), false), to_date(2022-12-31, None, Some(America/Los_Angeles), false)) AS (NOT between(to_timestamp(2022-12-26 00:00:01), to_date(2022-03-01), to_date(2022-12-31)))#x] +Project [NOT between(to_timestamp(2022-12-26 00:00:01, None, TimestampType, Some(America/Los_Angeles), true), to_date(2022-03-01, None, Some(America/Los_Angeles), true), to_date(2022-12-31, None, Some(America/Los_Angeles), true)) AS (NOT between(to_timestamp(2022-12-26 00:00:01), to_date(2022-03-01), to_date(2022-12-31)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subexp-elimination.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subexp-elimination.sql.out index 754b05bfa6fed..41807814adfb6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subexp-elimination.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subexp-elimination.sql.out @@ -51,7 +51,7 @@ Project [if (isnull(from_json(StructField(a,IntegerType,true), StructField(b,Str -- !query SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 else from_json(a, 'struct').b + 2 end FROM testData -- !query analysis -Project [CASE WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).a > 5) THEN from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).a > 4) THEN cast((cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b as double) + cast(1 as double)) as string) ELSE cast((cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b as double) + cast(2 as double)) as string) END AS CASE WHEN (from_json(a).a > 5) THEN from_json(a).b WHEN (from_json(a).a > 4) THEN (from_json(a).b + 1) ELSE (from_json(a).b + 2) END#x] +Project [CASE WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).a > 5) THEN cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b as bigint) WHEN (from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).a > 4) THEN (cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b as bigint) + cast(1 as bigint)) ELSE (cast(from_json(StructField(a,IntegerType,true), StructField(b,StringType,true), a#x, Some(America/Los_Angeles), false).b as bigint) + cast(2 as bigint)) END AS CASE WHEN (from_json(a).a > 5) THEN from_json(a).b WHEN (from_json(a).a > 4) THEN (from_json(a).b + 1) ELSE (from_json(a).b + 2) END#xL] +- SubqueryAlias testdata +- View (`testData`, [a#x, b#x]) +- Project [cast(a#x as string) AS a#x, cast(b#x as string) AS b#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out index 4ff0222d6e965..f8f23009258ab 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -315,7 +315,7 @@ WHERE t1c + 5 = (SELECT max(t2e) FROM t2) -- !query analysis Project [t1a#x, t1b#x, t1g#x] -+- Filter (cast((t1c#x + 5) as float) = scalar-subquery#x []) ++- Filter (cast((t1c#x + 5) as double) = cast(scalar-subquery#x [] as double)) : +- Aggregate [max(t2e#x) AS max(t2e)#x] : +- SubqueryAlias t2 : +- View (`t2`, [t2a#x, t2b#x, t2c#x, t2d#xL, t2e#x, t2f#x, t2g#x, t2h#x, t2i#x]) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ltz.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ltz.sql.out index c371876fde6a6..039556b59b0ba 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ltz.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ltz.sql.out @@ -8,42 +8,42 @@ select timestamp_ltz'2016-12-31 00:12:00', timestamp_ltz'2016-12-31' -- !query select to_timestamp_ltz(null), to_timestamp_ltz('2016-12-31 00:12:00'), to_timestamp_ltz('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_timestamp_ltz(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(NULL)#x, to_timestamp_ltz(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(2016-12-31 00:12:00)#x, to_timestamp_ltz(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(2016-12-31, yyyy-MM-dd)#x] +Project [to_timestamp_ltz(cast(null as string), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(NULL)#x, to_timestamp_ltz(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(2016-12-31 00:12:00)#x, to_timestamp_ltz(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp_ltz(to_date(null)), to_timestamp_ltz(to_date('2016-12-31')) -- !query analysis -Project [to_timestamp_ltz(to_date(cast(null as string), None, Some(America/Los_Angeles), false), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(to_date(NULL))#x, to_timestamp_ltz(to_date(2016-12-31, None, Some(America/Los_Angeles), false), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(to_date(2016-12-31))#x] +Project [to_timestamp_ltz(to_date(cast(null as string), None, Some(America/Los_Angeles), true), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(to_date(NULL))#x, to_timestamp_ltz(to_date(2016-12-31, None, Some(America/Los_Angeles), true), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(to_date(2016-12-31))#x] +- OneRowRelation -- !query select to_timestamp_ltz(to_timestamp_ntz(null)), to_timestamp_ltz(to_timestamp_ntz('2016-12-31 00:12:00')) -- !query analysis -Project [to_timestamp_ltz(to_timestamp_ntz(cast(null as string), None, TimestampNTZType, Some(America/Los_Angeles), false), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(to_timestamp_ntz(NULL))#x, to_timestamp_ltz(to_timestamp_ntz(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), false), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp_ltz(to_timestamp_ntz(2016-12-31 00:12:00))#x] +Project [to_timestamp_ltz(to_timestamp_ntz(cast(null as string), None, TimestampNTZType, Some(America/Los_Angeles), true), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(to_timestamp_ntz(NULL))#x, to_timestamp_ltz(to_timestamp_ntz(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), true), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp_ltz(to_timestamp_ntz(2016-12-31 00:12:00))#x] +- OneRowRelation -- !query SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678) -- !query analysis -Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 45.678)#x] +Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 45.678)#x] +- OneRowRelation -- !query SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678, 'CET') -- !query analysis -Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 45.678, CET)#x] +Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), true, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 45.678, CET)#x] +- OneRowRelation -- !query SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007) -- !query analysis -Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 60.007)#x] +Project [make_timestamp_ltz(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp_ltz(2021, 7, 11, 6, 30, 60.007)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ntz.sql.out index c285169c572e5..e92a392e22b67 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp-ntz.sql.out @@ -9,28 +9,28 @@ Project [2016-12-31 00:12:00 AS TIMESTAMP_NTZ '2016-12-31 00:12:00'#x, 2016-12-3 -- !query select to_timestamp_ntz(null), to_timestamp_ntz('2016-12-31 00:12:00'), to_timestamp_ntz('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_timestamp_ntz(cast(null as string), None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(NULL)#x, to_timestamp_ntz(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(2016-12-31 00:12:00)#x, to_timestamp_ntz(2016-12-31, Some(yyyy-MM-dd), TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(2016-12-31, yyyy-MM-dd)#x] +Project [to_timestamp_ntz(cast(null as string), None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(NULL)#x, to_timestamp_ntz(2016-12-31 00:12:00, None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(2016-12-31 00:12:00)#x, to_timestamp_ntz(2016-12-31, Some(yyyy-MM-dd), TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp_ntz(to_date(null)), to_timestamp_ntz(to_date('2016-12-31')) -- !query analysis -Project [to_timestamp_ntz(to_date(cast(null as string), None, Some(America/Los_Angeles), false), None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(to_date(NULL))#x, to_timestamp_ntz(to_date(2016-12-31, None, Some(America/Los_Angeles), false), None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(to_date(2016-12-31))#x] +Project [to_timestamp_ntz(to_date(cast(null as string), None, Some(America/Los_Angeles), true), None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(to_date(NULL))#x, to_timestamp_ntz(to_date(2016-12-31, None, Some(America/Los_Angeles), true), None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(to_date(2016-12-31))#x] +- OneRowRelation -- !query select to_timestamp_ntz(to_timestamp_ltz(null)), to_timestamp_ntz(to_timestamp_ltz('2016-12-31 00:12:00')) -- !query analysis -Project [to_timestamp_ntz(to_timestamp_ltz(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false), None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(to_timestamp_ltz(NULL))#x, to_timestamp_ntz(to_timestamp_ltz(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false), None, TimestampNTZType, Some(America/Los_Angeles), false) AS to_timestamp_ntz(to_timestamp_ltz(2016-12-31 00:12:00))#x] +Project [to_timestamp_ntz(to_timestamp_ltz(cast(null as string), None, TimestampType, Some(America/Los_Angeles), true), None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(to_timestamp_ltz(NULL))#x, to_timestamp_ntz(to_timestamp_ltz(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), true), None, TimestampNTZType, Some(America/Los_Angeles), true) AS to_timestamp_ntz(to_timestamp_ltz(2016-12-31 00:12:00))#x] +- OneRowRelation -- !query SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678) -- !query analysis -Project [make_timestamp_ntz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampNTZType) AS make_timestamp_ntz(2021, 7, 11, 6, 30, 45.678)#x] +Project [make_timestamp_ntz(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampNTZType) AS make_timestamp_ntz(2021, 7, 11, 6, 30, 45.678)#x] +- OneRowRelation @@ -60,7 +60,7 @@ org.apache.spark.sql.AnalysisException -- !query SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 60.007) -- !query analysis -Project [make_timestamp_ntz(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampNTZType) AS make_timestamp_ntz(2021, 7, 11, 6, 30, 60.007)#x] +Project [make_timestamp_ntz(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampNTZType) AS make_timestamp_ntz(2021, 7, 11, 6, 30, 60.007)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out index dcfd783b648f8..560974d28c545 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timestamp.sql.out @@ -90,70 +90,70 @@ Project [(localtimestamp(Some(America/Los_Angeles)) = localtimestamp(Some(Americ -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(45.678 as decimal(16,6)), Some(CET), Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 45.678, CET)#x] +- OneRowRelation -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query analysis -Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +Project [make_timestamp(2021, 7, 11, 6, 30, cast(60.007 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(2021, 7, 11, 6, 30, 60.007)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 1) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(1 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 1)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 60) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(60 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 60)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(61 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 61)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, null) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(null as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, NULL)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(59.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 59.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(99.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 99.999999)#x] +- OneRowRelation -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query analysis -Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), false, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +Project [make_timestamp(1, 1, 1, 1, 1, cast(999.999999 as decimal(16,6)), None, Some(America/Los_Angeles), true, TimestampType) AS make_timestamp(1, 1, 1, 1, 1, 999.999999)#x] +- OneRowRelation @@ -302,231 +302,231 @@ select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020- -- !query select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') -- !query analysis -Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +Project [to_timestamp(cast(null as string), None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(NULL)#x, to_timestamp(2016-12-31 00:12:00, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31 00:12:00)#x, to_timestamp(2016-12-31, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2016-12-31, yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp(1) -- !query analysis -Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(1)#x] +Project [to_timestamp(1, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(1)#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12., Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.0, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.0, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123UTC, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123UTC, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.12345CST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12345CST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.123456PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123456PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234567PST, Some(yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234567PST, yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(123456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(123456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query analysis -Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +Project [to_timestamp(223456 2019-10-06 10:11:12.123456PST, Some(SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(223456 2019-10-06 10:11:12.123456PST, SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.1234, Some(yyyy-MM-dd HH:mm:ss.[SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.1234, yyyy-MM-dd HH:mm:ss.[SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.123, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.123, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12, Some(yyyy-MM-dd HH:mm:ss[.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12, yyyy-MM-dd HH:mm:ss[.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11:12.12, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11:12.12, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query analysis -Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +Project [to_timestamp(2019-10-06 10:11, Some(yyyy-MM-dd HH:mm[:ss.SSSSSS]), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 10:11, yyyy-MM-dd HH:mm[:ss.SSSSSS])#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") -- !query analysis -Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +Project [to_timestamp(2019-10-06S10:11:12.12345, Some(yyyy-MM-dd'S'HH:mm:ss.SSSSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S10:11:12.12345, yyyy-MM-dd'S'HH:mm:ss.SSSSSS)#x] +- OneRowRelation -- !query select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.12342019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.12342019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyyyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyyyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1232019-10-06S10:11, Some(ss.SSSSyy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1232019-10-06S10:11, ss.SSSSyy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") -- !query analysis -Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +Project [to_timestamp(12.1234019-10-06S10:11, Some(ss.SSSSy-MM-dd'S'HH:mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(12.1234019-10-06S10:11, ss.SSSSy-MM-dd'S'HH:mm)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") -- !query analysis -Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +Project [to_timestamp(2019-10-06S, Some(yyyy-MM-dd'S'), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06S, yyyy-MM-dd'S')#x] +- OneRowRelation -- !query select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") -- !query analysis -Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +Project [to_timestamp(S2019-10-06, Some('S'yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(S2019-10-06, 'S'yyyy-MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +Project [to_timestamp(2019-10-06T10:11:12'12, Some(yyyy-MM-dd'T'HH:mm:ss''SSSS), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12'12, yyyy-MM-dd'T'HH:mm:ss''SSSS)#x] +- OneRowRelation -- !query select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") -- !query analysis -Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +Project [to_timestamp(2019-10-06T10:11:12', Some(yyyy-MM-dd'T'HH:mm:ss''), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06T10:11:12', yyyy-MM-dd'T'HH:mm:ss'')#x] +- OneRowRelation -- !query select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp('2019-10-06T10:11:12, Some(''yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp('2019-10-06T10:11:12, ''yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") -- !query analysis -Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +Project [to_timestamp(P2019-10-06T10:11:12, Some('P'yyyy-MM-dd'T'HH:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(P2019-10-06T10:11:12, 'P'yyyy-MM-dd'T'HH:mm:ss)#x] +- OneRowRelation -- !query select to_timestamp("16", "dd") -- !query analysis -Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(16, dd)#x] +Project [to_timestamp(16, Some(dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(16, dd)#x] +- OneRowRelation -- !query select to_timestamp("02-29", "MM-dd") -- !query analysis -Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(02-29, MM-dd)#x] +Project [to_timestamp(02-29, Some(MM-dd), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(02-29, MM-dd)#x] +- OneRowRelation -- !query select to_timestamp("2019 40", "yyyy mm") -- !query analysis -Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 40, yyyy mm)#x] +Project [to_timestamp(2019 40, Some(yyyy mm), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 40, yyyy mm)#x] +- OneRowRelation -- !query select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") -- !query analysis -Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +Project [to_timestamp(2019 10:10:10, Some(yyyy hh:mm:ss), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019 10:10:10, yyyy hh:mm:ss)#x] +- OneRowRelation @@ -551,49 +551,13 @@ select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:10\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:11\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -619,49 +583,13 @@ CreateViewCommand `ts_view`, select '2011-11-11 11:11:11' str, false, false, Loc -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "str - timestamp'2011-11-11 11:11:11'" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "timestamp'2011-11-11 11:11:11' - str" - } ] -} +[Analyzer test output redacted due to nondeterminism] -- !query @@ -669,11 +597,11 @@ select timestamp'2011-11-11 11:11:11' + '1' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -691,11 +619,11 @@ select '1' + timestamp'2011-11-11 11:11:11' -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { @@ -774,28 +702,28 @@ select date '2012-01-01' - interval 3 hours, -- !query select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') -- !query analysis -Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +Project [to_timestamp(2019-10-06 A, Some(yyyy-MM-dd GGGGG), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2019-10-06 A, yyyy-MM-dd GGGGG)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEEE)#x] +- OneRowRelation -- !query select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +Project [to_timestamp(22 05 2020 Friday, Some(dd MM yyyy EEEEE), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#x] +- OneRowRelation -- !query select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') -- !query analysis -Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), false) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +Project [unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE, Some(America/Los_Angeles), true) AS unix_timestamp(22 05 2020 Friday, dd MM yyyy EEEEE)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index 7cf8a2886069d..f0d3b8b999a29 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -786,8 +786,8 @@ FROM( SELECT (b + 1) AS result ORDER BY result -- !query analysis -Sort [result#x ASC NULLS FIRST], true -+- Project [(cast(b#x as double) + cast(1 as double)) AS result#x] +Sort [result#xL ASC NULLS FIRST], true ++- Project [(cast(b#x as bigint) + cast(1 as bigint)) AS result#xL] +- SubqueryAlias t +- ScriptTransformation cat, [a#x, b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) +- Aggregate [a#x], [a#x, sum(b#x) AS sum(b)#xL] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out index caf997f6ccbb2..30654d1d71e2b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/try_arithmetic.sql.out @@ -23,7 +23,7 @@ Project [try_add(2147483647, cast(1 as decimal(10,0))) AS try_add(2147483647, 1) -- !query SELECT try_add(2147483647, "1") -- !query analysis -Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#x] +Project [try_add(2147483647, 1) AS try_add(2147483647, 1)#xL] +- OneRowRelation @@ -305,7 +305,7 @@ Project [try_subtract(2147483647, cast(-1 as decimal(10,0))) AS try_subtract(214 -- !query SELECT try_subtract(2147483647, "-1") -- !query analysis -Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#x] +Project [try_subtract(2147483647, -1) AS try_subtract(2147483647, -1)#xL] +- OneRowRelation @@ -403,7 +403,7 @@ Project [try_multiply(2147483647, cast(-2 as decimal(10,0))) AS try_multiply(214 -- !query SELECT try_multiply(2147483647, "-2") -- !query analysis -Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#x] +Project [try_multiply(2147483647, -2) AS try_multiply(2147483647, -2)#xL] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/binaryComparison.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/binaryComparison.sql.out index d15418c17b730..1395cc350db72 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/binaryComparison.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/binaryComparison.sql.out @@ -10,271 +10,583 @@ CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, UNSUPPORTED, true -- !query SELECT cast(1 as binary) = '1' FROM t -- !query analysis -Project [(cast(1 as binary) = cast(1 as binary)) AS (CAST(1 AS BINARY) = 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) > '2' FROM t -- !query analysis -Project [(cast(1 as binary) > cast(2 as binary)) AS (CAST(1 AS BINARY) > 2)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) >= '2' FROM t -- !query analysis -Project [(cast(1 as binary) >= cast(2 as binary)) AS (CAST(1 AS BINARY) >= 2)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) < '2' FROM t -- !query analysis -Project [(cast(1 as binary) < cast(2 as binary)) AS (CAST(1 AS BINARY) < 2)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <= '2' FROM t -- !query analysis -Project [(cast(1 as binary) <= cast(2 as binary)) AS (CAST(1 AS BINARY) <= 2)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as binary) = cast(2 as binary)) AS (NOT (CAST(1 AS BINARY) = 2))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as binary) = cast(cast(null as string) as binary)) AS (CAST(1 AS BINARY) = CAST(NULL AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as binary) > cast(cast(null as string) as binary)) AS (CAST(1 AS BINARY) > CAST(NULL AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as binary) >= cast(cast(null as string) as binary)) AS (CAST(1 AS BINARY) >= CAST(NULL AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as binary) < cast(cast(null as string) as binary)) AS (CAST(1 AS BINARY) < CAST(NULL AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as binary) <= cast(cast(null as string) as binary)) AS (CAST(1 AS BINARY) <= CAST(NULL AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as binary) = cast(cast(null as string) as binary)) AS (NOT (CAST(1 AS BINARY) = CAST(NULL AS STRING)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '1' = cast(1 as binary) FROM t -- !query analysis -Project [(cast(1 as binary) = cast(1 as binary)) AS (1 = CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' > cast(1 as binary) FROM t -- !query analysis -Project [(cast(2 as binary) > cast(1 as binary)) AS (2 > CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' >= cast(1 as binary) FROM t -- !query analysis -Project [(cast(2 as binary) >= cast(1 as binary)) AS (2 >= CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' < cast(1 as binary) FROM t -- !query analysis -Project [(cast(2 as binary) < cast(1 as binary)) AS (2 < CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' <= cast(1 as binary) FROM t -- !query analysis -Project [(cast(2 as binary) <= cast(1 as binary)) AS (2 <= CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' <> cast(1 as binary) FROM t -- !query analysis -Project [NOT (cast(2 as binary) = cast(1 as binary)) AS (NOT (2 = CAST(1 AS BINARY)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) = cast(1 as binary) FROM t -- !query analysis -Project [(cast(cast(null as string) as binary) = cast(1 as binary)) AS (CAST(NULL AS STRING) = CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) > cast(1 as binary) FROM t -- !query analysis -Project [(cast(cast(null as string) as binary) > cast(1 as binary)) AS (CAST(NULL AS STRING) > CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) >= cast(1 as binary) FROM t -- !query analysis -Project [(cast(cast(null as string) as binary) >= cast(1 as binary)) AS (CAST(NULL AS STRING) >= CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) < cast(1 as binary) FROM t -- !query analysis -Project [(cast(cast(null as string) as binary) < cast(1 as binary)) AS (CAST(NULL AS STRING) < CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) <= cast(1 as binary) FROM t -- !query analysis -Project [(cast(cast(null as string) as binary) <= cast(1 as binary)) AS (CAST(NULL AS STRING) <= CAST(1 AS BINARY))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) <> cast(1 as binary) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as binary) = cast(1 as binary)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS BINARY)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as tinyint) = '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (CAST(1 AS TINYINT) = 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) = cast(1 as bigint)) AS (CAST(1 AS TINYINT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -285,7 +597,7 @@ Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (CAST(1 AS TINYINT) = 1)#x -- !query SELECT cast(1 as tinyint) > '2' FROM t -- !query analysis -Project [(cast(1 as tinyint) > cast(2 as tinyint)) AS (CAST(1 AS TINYINT) > 2)#x] +Project [(cast(cast(1 as tinyint) as bigint) > cast(2 as bigint)) AS (CAST(1 AS TINYINT) > 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -296,7 +608,7 @@ Project [(cast(1 as tinyint) > cast(2 as tinyint)) AS (CAST(1 AS TINYINT) > 2)#x -- !query SELECT cast(1 as tinyint) >= '2' FROM t -- !query analysis -Project [(cast(1 as tinyint) >= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) >= 2)#x] +Project [(cast(cast(1 as tinyint) as bigint) >= cast(2 as bigint)) AS (CAST(1 AS TINYINT) >= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -307,7 +619,7 @@ Project [(cast(1 as tinyint) >= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) >= 2) -- !query SELECT cast(1 as tinyint) < '2' FROM t -- !query analysis -Project [(cast(1 as tinyint) < cast(2 as tinyint)) AS (CAST(1 AS TINYINT) < 2)#x] +Project [(cast(cast(1 as tinyint) as bigint) < cast(2 as bigint)) AS (CAST(1 AS TINYINT) < 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -318,7 +630,7 @@ Project [(cast(1 as tinyint) < cast(2 as tinyint)) AS (CAST(1 AS TINYINT) < 2)#x -- !query SELECT cast(1 as tinyint) <= '2' FROM t -- !query analysis -Project [(cast(1 as tinyint) <= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) <= 2)#x] +Project [(cast(cast(1 as tinyint) as bigint) <= cast(2 as bigint)) AS (CAST(1 AS TINYINT) <= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -329,7 +641,7 @@ Project [(cast(1 as tinyint) <= cast(2 as tinyint)) AS (CAST(1 AS TINYINT) <= 2) -- !query SELECT cast(1 as tinyint) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as tinyint) = cast(2 as tinyint)) AS (NOT (CAST(1 AS TINYINT) = 2))#x] +Project [NOT (cast(cast(1 as tinyint) as bigint) = cast(2 as bigint)) AS (NOT (CAST(1 AS TINYINT) = 2))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -340,7 +652,7 @@ Project [NOT (cast(1 as tinyint) = cast(2 as tinyint)) AS (NOT (CAST(1 AS TINYIN -- !query SELECT cast(1 as tinyint) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(cast(null as string) as tinyint)) AS (CAST(1 AS TINYINT) = CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as tinyint) as bigint) = cast(cast(null as string) as bigint)) AS (CAST(1 AS TINYINT) = CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -351,7 +663,7 @@ Project [(cast(1 as tinyint) = cast(cast(null as string) as tinyint)) AS (CAST(1 -- !query SELECT cast(1 as tinyint) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as tinyint) > cast(cast(null as string) as tinyint)) AS (CAST(1 AS TINYINT) > CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as tinyint) as bigint) > cast(cast(null as string) as bigint)) AS (CAST(1 AS TINYINT) > CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -362,7 +674,7 @@ Project [(cast(1 as tinyint) > cast(cast(null as string) as tinyint)) AS (CAST(1 -- !query SELECT cast(1 as tinyint) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as tinyint) >= cast(cast(null as string) as tinyint)) AS (CAST(1 AS TINYINT) >= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as tinyint) as bigint) >= cast(cast(null as string) as bigint)) AS (CAST(1 AS TINYINT) >= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -373,7 +685,7 @@ Project [(cast(1 as tinyint) >= cast(cast(null as string) as tinyint)) AS (CAST( -- !query SELECT cast(1 as tinyint) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as tinyint) < cast(cast(null as string) as tinyint)) AS (CAST(1 AS TINYINT) < CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as tinyint) as bigint) < cast(cast(null as string) as bigint)) AS (CAST(1 AS TINYINT) < CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -384,7 +696,7 @@ Project [(cast(1 as tinyint) < cast(cast(null as string) as tinyint)) AS (CAST(1 -- !query SELECT cast(1 as tinyint) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as tinyint) <= cast(cast(null as string) as tinyint)) AS (CAST(1 AS TINYINT) <= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as tinyint) as bigint) <= cast(cast(null as string) as bigint)) AS (CAST(1 AS TINYINT) <= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -395,7 +707,7 @@ Project [(cast(1 as tinyint) <= cast(cast(null as string) as tinyint)) AS (CAST( -- !query SELECT cast(1 as tinyint) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as tinyint) = cast(cast(null as string) as tinyint)) AS (NOT (CAST(1 AS TINYINT) = CAST(NULL AS STRING)))#x] +Project [NOT (cast(cast(1 as tinyint) as bigint) = cast(cast(null as string) as bigint)) AS (NOT (CAST(1 AS TINYINT) = CAST(NULL AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -406,7 +718,7 @@ Project [NOT (cast(1 as tinyint) = cast(cast(null as string) as tinyint)) AS (NO -- !query SELECT '1' = cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (1 = CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as tinyint) as bigint)) AS (1 = CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -417,7 +729,7 @@ Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (1 = CAST(1 AS TINYINT))#x -- !query SELECT '2' > cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(2 as tinyint) > cast(1 as tinyint)) AS (2 > CAST(1 AS TINYINT))#x] +Project [(cast(2 as bigint) > cast(cast(1 as tinyint) as bigint)) AS (2 > CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -428,7 +740,7 @@ Project [(cast(2 as tinyint) > cast(1 as tinyint)) AS (2 > CAST(1 AS TINYINT))#x -- !query SELECT '2' >= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(2 as tinyint) >= cast(1 as tinyint)) AS (2 >= CAST(1 AS TINYINT))#x] +Project [(cast(2 as bigint) >= cast(cast(1 as tinyint) as bigint)) AS (2 >= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -439,7 +751,7 @@ Project [(cast(2 as tinyint) >= cast(1 as tinyint)) AS (2 >= CAST(1 AS TINYINT)) -- !query SELECT '2' < cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(2 as tinyint) < cast(1 as tinyint)) AS (2 < CAST(1 AS TINYINT))#x] +Project [(cast(2 as bigint) < cast(cast(1 as tinyint) as bigint)) AS (2 < CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -450,7 +762,7 @@ Project [(cast(2 as tinyint) < cast(1 as tinyint)) AS (2 < CAST(1 AS TINYINT))#x -- !query SELECT '2' <= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(2 as tinyint) <= cast(1 as tinyint)) AS (2 <= CAST(1 AS TINYINT))#x] +Project [(cast(2 as bigint) <= cast(cast(1 as tinyint) as bigint)) AS (2 <= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -461,7 +773,7 @@ Project [(cast(2 as tinyint) <= cast(1 as tinyint)) AS (2 <= CAST(1 AS TINYINT)) -- !query SELECT '2' <> cast(1 as tinyint) FROM t -- !query analysis -Project [NOT (cast(2 as tinyint) = cast(1 as tinyint)) AS (NOT (2 = CAST(1 AS TINYINT)))#x] +Project [NOT (cast(2 as bigint) = cast(cast(1 as tinyint) as bigint)) AS (NOT (2 = CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -472,7 +784,7 @@ Project [NOT (cast(2 as tinyint) = cast(1 as tinyint)) AS (NOT (2 = CAST(1 AS TI -- !query SELECT cast(null as string) = cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(null as string) as tinyint) = cast(1 as tinyint)) AS (CAST(NULL AS STRING) = CAST(1 AS TINYINT))#x] +Project [(cast(cast(null as string) as bigint) = cast(cast(1 as tinyint) as bigint)) AS (CAST(NULL AS STRING) = CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -483,7 +795,7 @@ Project [(cast(cast(null as string) as tinyint) = cast(1 as tinyint)) AS (CAST(N -- !query SELECT cast(null as string) > cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(null as string) as tinyint) > cast(1 as tinyint)) AS (CAST(NULL AS STRING) > CAST(1 AS TINYINT))#x] +Project [(cast(cast(null as string) as bigint) > cast(cast(1 as tinyint) as bigint)) AS (CAST(NULL AS STRING) > CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -494,7 +806,7 @@ Project [(cast(cast(null as string) as tinyint) > cast(1 as tinyint)) AS (CAST(N -- !query SELECT cast(null as string) >= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(null as string) as tinyint) >= cast(1 as tinyint)) AS (CAST(NULL AS STRING) >= CAST(1 AS TINYINT))#x] +Project [(cast(cast(null as string) as bigint) >= cast(cast(1 as tinyint) as bigint)) AS (CAST(NULL AS STRING) >= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -505,7 +817,7 @@ Project [(cast(cast(null as string) as tinyint) >= cast(1 as tinyint)) AS (CAST( -- !query SELECT cast(null as string) < cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(null as string) as tinyint) < cast(1 as tinyint)) AS (CAST(NULL AS STRING) < CAST(1 AS TINYINT))#x] +Project [(cast(cast(null as string) as bigint) < cast(cast(1 as tinyint) as bigint)) AS (CAST(NULL AS STRING) < CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -516,7 +828,7 @@ Project [(cast(cast(null as string) as tinyint) < cast(1 as tinyint)) AS (CAST(N -- !query SELECT cast(null as string) <= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(null as string) as tinyint) <= cast(1 as tinyint)) AS (CAST(NULL AS STRING) <= CAST(1 AS TINYINT))#x] +Project [(cast(cast(null as string) as bigint) <= cast(cast(1 as tinyint) as bigint)) AS (CAST(NULL AS STRING) <= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -527,7 +839,7 @@ Project [(cast(cast(null as string) as tinyint) <= cast(1 as tinyint)) AS (CAST( -- !query SELECT cast(null as string) <> cast(1 as tinyint) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as tinyint) = cast(1 as tinyint)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS TINYINT)))#x] +Project [NOT (cast(cast(null as string) as bigint) = cast(cast(1 as tinyint) as bigint)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -538,7 +850,7 @@ Project [NOT (cast(cast(null as string) as tinyint) = cast(1 as tinyint)) AS (NO -- !query SELECT cast(1 as smallint) = '1' FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 1)#x] +Project [(cast(cast(1 as smallint) as bigint) = cast(1 as bigint)) AS (CAST(1 AS SMALLINT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -549,7 +861,7 @@ Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 1 -- !query SELECT cast(1 as smallint) > '2' FROM t -- !query analysis -Project [(cast(1 as smallint) > cast(2 as smallint)) AS (CAST(1 AS SMALLINT) > 2)#x] +Project [(cast(cast(1 as smallint) as bigint) > cast(2 as bigint)) AS (CAST(1 AS SMALLINT) > 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -560,7 +872,7 @@ Project [(cast(1 as smallint) > cast(2 as smallint)) AS (CAST(1 AS SMALLINT) > 2 -- !query SELECT cast(1 as smallint) >= '2' FROM t -- !query analysis -Project [(cast(1 as smallint) >= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) >= 2)#x] +Project [(cast(cast(1 as smallint) as bigint) >= cast(2 as bigint)) AS (CAST(1 AS SMALLINT) >= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -571,7 +883,7 @@ Project [(cast(1 as smallint) >= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) >= -- !query SELECT cast(1 as smallint) < '2' FROM t -- !query analysis -Project [(cast(1 as smallint) < cast(2 as smallint)) AS (CAST(1 AS SMALLINT) < 2)#x] +Project [(cast(cast(1 as smallint) as bigint) < cast(2 as bigint)) AS (CAST(1 AS SMALLINT) < 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -582,7 +894,7 @@ Project [(cast(1 as smallint) < cast(2 as smallint)) AS (CAST(1 AS SMALLINT) < 2 -- !query SELECT cast(1 as smallint) <= '2' FROM t -- !query analysis -Project [(cast(1 as smallint) <= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) <= 2)#x] +Project [(cast(cast(1 as smallint) as bigint) <= cast(2 as bigint)) AS (CAST(1 AS SMALLINT) <= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -593,7 +905,7 @@ Project [(cast(1 as smallint) <= cast(2 as smallint)) AS (CAST(1 AS SMALLINT) <= -- !query SELECT cast(1 as smallint) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as smallint) = cast(2 as smallint)) AS (NOT (CAST(1 AS SMALLINT) = 2))#x] +Project [NOT (cast(cast(1 as smallint) as bigint) = cast(2 as bigint)) AS (NOT (CAST(1 AS SMALLINT) = 2))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -604,7 +916,7 @@ Project [NOT (cast(1 as smallint) = cast(2 as smallint)) AS (NOT (CAST(1 AS SMAL -- !query SELECT cast(1 as smallint) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(cast(null as string) as smallint)) AS (CAST(1 AS SMALLINT) = CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as smallint) as bigint) = cast(cast(null as string) as bigint)) AS (CAST(1 AS SMALLINT) = CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -615,7 +927,7 @@ Project [(cast(1 as smallint) = cast(cast(null as string) as smallint)) AS (CAST -- !query SELECT cast(1 as smallint) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as smallint) > cast(cast(null as string) as smallint)) AS (CAST(1 AS SMALLINT) > CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as smallint) as bigint) > cast(cast(null as string) as bigint)) AS (CAST(1 AS SMALLINT) > CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -626,7 +938,7 @@ Project [(cast(1 as smallint) > cast(cast(null as string) as smallint)) AS (CAST -- !query SELECT cast(1 as smallint) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as smallint) >= cast(cast(null as string) as smallint)) AS (CAST(1 AS SMALLINT) >= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as smallint) as bigint) >= cast(cast(null as string) as bigint)) AS (CAST(1 AS SMALLINT) >= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -637,7 +949,7 @@ Project [(cast(1 as smallint) >= cast(cast(null as string) as smallint)) AS (CAS -- !query SELECT cast(1 as smallint) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as smallint) < cast(cast(null as string) as smallint)) AS (CAST(1 AS SMALLINT) < CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as smallint) as bigint) < cast(cast(null as string) as bigint)) AS (CAST(1 AS SMALLINT) < CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -648,7 +960,7 @@ Project [(cast(1 as smallint) < cast(cast(null as string) as smallint)) AS (CAST -- !query SELECT cast(1 as smallint) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as smallint) <= cast(cast(null as string) as smallint)) AS (CAST(1 AS SMALLINT) <= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as smallint) as bigint) <= cast(cast(null as string) as bigint)) AS (CAST(1 AS SMALLINT) <= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -659,7 +971,7 @@ Project [(cast(1 as smallint) <= cast(cast(null as string) as smallint)) AS (CAS -- !query SELECT cast(1 as smallint) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as smallint) = cast(cast(null as string) as smallint)) AS (NOT (CAST(1 AS SMALLINT) = CAST(NULL AS STRING)))#x] +Project [NOT (cast(cast(1 as smallint) as bigint) = cast(cast(null as string) as bigint)) AS (NOT (CAST(1 AS SMALLINT) = CAST(NULL AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -670,7 +982,7 @@ Project [NOT (cast(1 as smallint) = cast(cast(null as string) as smallint)) AS ( -- !query SELECT '1' = cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(1 as smallint)) AS (1 = CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as smallint) as bigint)) AS (1 = CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -681,7 +993,7 @@ Project [(cast(1 as smallint) = cast(1 as smallint)) AS (1 = CAST(1 AS SMALLINT) -- !query SELECT '2' > cast(1 as smallint) FROM t -- !query analysis -Project [(cast(2 as smallint) > cast(1 as smallint)) AS (2 > CAST(1 AS SMALLINT))#x] +Project [(cast(2 as bigint) > cast(cast(1 as smallint) as bigint)) AS (2 > CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -692,7 +1004,7 @@ Project [(cast(2 as smallint) > cast(1 as smallint)) AS (2 > CAST(1 AS SMALLINT) -- !query SELECT '2' >= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(2 as smallint) >= cast(1 as smallint)) AS (2 >= CAST(1 AS SMALLINT))#x] +Project [(cast(2 as bigint) >= cast(cast(1 as smallint) as bigint)) AS (2 >= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -703,7 +1015,7 @@ Project [(cast(2 as smallint) >= cast(1 as smallint)) AS (2 >= CAST(1 AS SMALLIN -- !query SELECT '2' < cast(1 as smallint) FROM t -- !query analysis -Project [(cast(2 as smallint) < cast(1 as smallint)) AS (2 < CAST(1 AS SMALLINT))#x] +Project [(cast(2 as bigint) < cast(cast(1 as smallint) as bigint)) AS (2 < CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -714,7 +1026,7 @@ Project [(cast(2 as smallint) < cast(1 as smallint)) AS (2 < CAST(1 AS SMALLINT) -- !query SELECT '2' <= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(2 as smallint) <= cast(1 as smallint)) AS (2 <= CAST(1 AS SMALLINT))#x] +Project [(cast(2 as bigint) <= cast(cast(1 as smallint) as bigint)) AS (2 <= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -725,7 +1037,7 @@ Project [(cast(2 as smallint) <= cast(1 as smallint)) AS (2 <= CAST(1 AS SMALLIN -- !query SELECT '2' <> cast(1 as smallint) FROM t -- !query analysis -Project [NOT (cast(2 as smallint) = cast(1 as smallint)) AS (NOT (2 = CAST(1 AS SMALLINT)))#x] +Project [NOT (cast(2 as bigint) = cast(cast(1 as smallint) as bigint)) AS (NOT (2 = CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -736,7 +1048,7 @@ Project [NOT (cast(2 as smallint) = cast(1 as smallint)) AS (NOT (2 = CAST(1 AS -- !query SELECT cast(null as string) = cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(null as string) as smallint) = cast(1 as smallint)) AS (CAST(NULL AS STRING) = CAST(1 AS SMALLINT))#x] +Project [(cast(cast(null as string) as bigint) = cast(cast(1 as smallint) as bigint)) AS (CAST(NULL AS STRING) = CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -747,7 +1059,7 @@ Project [(cast(cast(null as string) as smallint) = cast(1 as smallint)) AS (CAST -- !query SELECT cast(null as string) > cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(null as string) as smallint) > cast(1 as smallint)) AS (CAST(NULL AS STRING) > CAST(1 AS SMALLINT))#x] +Project [(cast(cast(null as string) as bigint) > cast(cast(1 as smallint) as bigint)) AS (CAST(NULL AS STRING) > CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -758,7 +1070,7 @@ Project [(cast(cast(null as string) as smallint) > cast(1 as smallint)) AS (CAST -- !query SELECT cast(null as string) >= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(null as string) as smallint) >= cast(1 as smallint)) AS (CAST(NULL AS STRING) >= CAST(1 AS SMALLINT))#x] +Project [(cast(cast(null as string) as bigint) >= cast(cast(1 as smallint) as bigint)) AS (CAST(NULL AS STRING) >= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -769,7 +1081,7 @@ Project [(cast(cast(null as string) as smallint) >= cast(1 as smallint)) AS (CAS -- !query SELECT cast(null as string) < cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(null as string) as smallint) < cast(1 as smallint)) AS (CAST(NULL AS STRING) < CAST(1 AS SMALLINT))#x] +Project [(cast(cast(null as string) as bigint) < cast(cast(1 as smallint) as bigint)) AS (CAST(NULL AS STRING) < CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -780,7 +1092,7 @@ Project [(cast(cast(null as string) as smallint) < cast(1 as smallint)) AS (CAST -- !query SELECT cast(null as string) <= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(null as string) as smallint) <= cast(1 as smallint)) AS (CAST(NULL AS STRING) <= CAST(1 AS SMALLINT))#x] +Project [(cast(cast(null as string) as bigint) <= cast(cast(1 as smallint) as bigint)) AS (CAST(NULL AS STRING) <= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -791,7 +1103,7 @@ Project [(cast(cast(null as string) as smallint) <= cast(1 as smallint)) AS (CAS -- !query SELECT cast(null as string) <> cast(1 as smallint) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as smallint) = cast(1 as smallint)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS SMALLINT)))#x] +Project [NOT (cast(cast(null as string) as bigint) = cast(cast(1 as smallint) as bigint)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -802,7 +1114,7 @@ Project [NOT (cast(cast(null as string) as smallint) = cast(1 as smallint)) AS ( -- !query SELECT cast(1 as int) = '1' FROM t -- !query analysis -Project [(cast(1 as int) = cast(1 as int)) AS (CAST(1 AS INT) = 1)#x] +Project [(cast(cast(1 as int) as bigint) = cast(1 as bigint)) AS (CAST(1 AS INT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -813,7 +1125,7 @@ Project [(cast(1 as int) = cast(1 as int)) AS (CAST(1 AS INT) = 1)#x] -- !query SELECT cast(1 as int) > '2' FROM t -- !query analysis -Project [(cast(1 as int) > cast(2 as int)) AS (CAST(1 AS INT) > 2)#x] +Project [(cast(cast(1 as int) as bigint) > cast(2 as bigint)) AS (CAST(1 AS INT) > 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -824,7 +1136,7 @@ Project [(cast(1 as int) > cast(2 as int)) AS (CAST(1 AS INT) > 2)#x] -- !query SELECT cast(1 as int) >= '2' FROM t -- !query analysis -Project [(cast(1 as int) >= cast(2 as int)) AS (CAST(1 AS INT) >= 2)#x] +Project [(cast(cast(1 as int) as bigint) >= cast(2 as bigint)) AS (CAST(1 AS INT) >= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -835,7 +1147,7 @@ Project [(cast(1 as int) >= cast(2 as int)) AS (CAST(1 AS INT) >= 2)#x] -- !query SELECT cast(1 as int) < '2' FROM t -- !query analysis -Project [(cast(1 as int) < cast(2 as int)) AS (CAST(1 AS INT) < 2)#x] +Project [(cast(cast(1 as int) as bigint) < cast(2 as bigint)) AS (CAST(1 AS INT) < 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -846,7 +1158,7 @@ Project [(cast(1 as int) < cast(2 as int)) AS (CAST(1 AS INT) < 2)#x] -- !query SELECT cast(1 as int) <= '2' FROM t -- !query analysis -Project [(cast(1 as int) <= cast(2 as int)) AS (CAST(1 AS INT) <= 2)#x] +Project [(cast(cast(1 as int) as bigint) <= cast(2 as bigint)) AS (CAST(1 AS INT) <= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -857,7 +1169,7 @@ Project [(cast(1 as int) <= cast(2 as int)) AS (CAST(1 AS INT) <= 2)#x] -- !query SELECT cast(1 as int) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as int) = cast(2 as int)) AS (NOT (CAST(1 AS INT) = 2))#x] +Project [NOT (cast(cast(1 as int) as bigint) = cast(2 as bigint)) AS (NOT (CAST(1 AS INT) = 2))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -868,7 +1180,7 @@ Project [NOT (cast(1 as int) = cast(2 as int)) AS (NOT (CAST(1 AS INT) = 2))#x] -- !query SELECT cast(1 as int) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as int) = cast(cast(null as string) as int)) AS (CAST(1 AS INT) = CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as int) as bigint) = cast(cast(null as string) as bigint)) AS (CAST(1 AS INT) = CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -879,7 +1191,7 @@ Project [(cast(1 as int) = cast(cast(null as string) as int)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as int) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as int) > cast(cast(null as string) as int)) AS (CAST(1 AS INT) > CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as int) as bigint) > cast(cast(null as string) as bigint)) AS (CAST(1 AS INT) > CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -890,7 +1202,7 @@ Project [(cast(1 as int) > cast(cast(null as string) as int)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as int) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as int) >= cast(cast(null as string) as int)) AS (CAST(1 AS INT) >= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as int) as bigint) >= cast(cast(null as string) as bigint)) AS (CAST(1 AS INT) >= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -901,7 +1213,7 @@ Project [(cast(1 as int) >= cast(cast(null as string) as int)) AS (CAST(1 AS INT -- !query SELECT cast(1 as int) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as int) < cast(cast(null as string) as int)) AS (CAST(1 AS INT) < CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as int) as bigint) < cast(cast(null as string) as bigint)) AS (CAST(1 AS INT) < CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -912,7 +1224,7 @@ Project [(cast(1 as int) < cast(cast(null as string) as int)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as int) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as int) <= cast(cast(null as string) as int)) AS (CAST(1 AS INT) <= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as int) as bigint) <= cast(cast(null as string) as bigint)) AS (CAST(1 AS INT) <= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -923,7 +1235,7 @@ Project [(cast(1 as int) <= cast(cast(null as string) as int)) AS (CAST(1 AS INT -- !query SELECT cast(1 as int) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as int) = cast(cast(null as string) as int)) AS (NOT (CAST(1 AS INT) = CAST(NULL AS STRING)))#x] +Project [NOT (cast(cast(1 as int) as bigint) = cast(cast(null as string) as bigint)) AS (NOT (CAST(1 AS INT) = CAST(NULL AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -934,7 +1246,7 @@ Project [NOT (cast(1 as int) = cast(cast(null as string) as int)) AS (NOT (CAST( -- !query SELECT '1' = cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) = cast(1 as int)) AS (1 = CAST(1 AS INT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as int) as bigint)) AS (1 = CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -945,7 +1257,7 @@ Project [(cast(1 as int) = cast(1 as int)) AS (1 = CAST(1 AS INT))#x] -- !query SELECT '2' > cast(1 as int) FROM t -- !query analysis -Project [(cast(2 as int) > cast(1 as int)) AS (2 > CAST(1 AS INT))#x] +Project [(cast(2 as bigint) > cast(cast(1 as int) as bigint)) AS (2 > CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -956,7 +1268,7 @@ Project [(cast(2 as int) > cast(1 as int)) AS (2 > CAST(1 AS INT))#x] -- !query SELECT '2' >= cast(1 as int) FROM t -- !query analysis -Project [(cast(2 as int) >= cast(1 as int)) AS (2 >= CAST(1 AS INT))#x] +Project [(cast(2 as bigint) >= cast(cast(1 as int) as bigint)) AS (2 >= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -967,7 +1279,7 @@ Project [(cast(2 as int) >= cast(1 as int)) AS (2 >= CAST(1 AS INT))#x] -- !query SELECT '2' < cast(1 as int) FROM t -- !query analysis -Project [(cast(2 as int) < cast(1 as int)) AS (2 < CAST(1 AS INT))#x] +Project [(cast(2 as bigint) < cast(cast(1 as int) as bigint)) AS (2 < CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -978,7 +1290,7 @@ Project [(cast(2 as int) < cast(1 as int)) AS (2 < CAST(1 AS INT))#x] -- !query SELECT '2' <> cast(1 as int) FROM t -- !query analysis -Project [NOT (cast(2 as int) = cast(1 as int)) AS (NOT (2 = CAST(1 AS INT)))#x] +Project [NOT (cast(2 as bigint) = cast(cast(1 as int) as bigint)) AS (NOT (2 = CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -989,7 +1301,7 @@ Project [NOT (cast(2 as int) = cast(1 as int)) AS (NOT (2 = CAST(1 AS INT)))#x] -- !query SELECT '2' <= cast(1 as int) FROM t -- !query analysis -Project [(cast(2 as int) <= cast(1 as int)) AS (2 <= CAST(1 AS INT))#x] +Project [(cast(2 as bigint) <= cast(cast(1 as int) as bigint)) AS (2 <= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1000,7 +1312,7 @@ Project [(cast(2 as int) <= cast(1 as int)) AS (2 <= CAST(1 AS INT))#x] -- !query SELECT cast(null as string) = cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(null as string) as int) = cast(1 as int)) AS (CAST(NULL AS STRING) = CAST(1 AS INT))#x] +Project [(cast(cast(null as string) as bigint) = cast(cast(1 as int) as bigint)) AS (CAST(NULL AS STRING) = CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1011,7 +1323,7 @@ Project [(cast(cast(null as string) as int) = cast(1 as int)) AS (CAST(NULL AS S -- !query SELECT cast(null as string) > cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(null as string) as int) > cast(1 as int)) AS (CAST(NULL AS STRING) > CAST(1 AS INT))#x] +Project [(cast(cast(null as string) as bigint) > cast(cast(1 as int) as bigint)) AS (CAST(NULL AS STRING) > CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1022,7 +1334,7 @@ Project [(cast(cast(null as string) as int) > cast(1 as int)) AS (CAST(NULL AS S -- !query SELECT cast(null as string) >= cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(null as string) as int) >= cast(1 as int)) AS (CAST(NULL AS STRING) >= CAST(1 AS INT))#x] +Project [(cast(cast(null as string) as bigint) >= cast(cast(1 as int) as bigint)) AS (CAST(NULL AS STRING) >= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1033,7 +1345,7 @@ Project [(cast(cast(null as string) as int) >= cast(1 as int)) AS (CAST(NULL AS -- !query SELECT cast(null as string) < cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(null as string) as int) < cast(1 as int)) AS (CAST(NULL AS STRING) < CAST(1 AS INT))#x] +Project [(cast(cast(null as string) as bigint) < cast(cast(1 as int) as bigint)) AS (CAST(NULL AS STRING) < CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1044,7 +1356,7 @@ Project [(cast(cast(null as string) as int) < cast(1 as int)) AS (CAST(NULL AS S -- !query SELECT cast(null as string) <> cast(1 as int) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as int) = cast(1 as int)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS INT)))#x] +Project [NOT (cast(cast(null as string) as bigint) = cast(cast(1 as int) as bigint)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1055,7 +1367,7 @@ Project [NOT (cast(cast(null as string) as int) = cast(1 as int)) AS (NOT (CAST( -- !query SELECT cast(null as string) <= cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(null as string) as int) <= cast(1 as int)) AS (CAST(NULL AS STRING) <= CAST(1 AS INT))#x] +Project [(cast(cast(null as string) as bigint) <= cast(cast(1 as int) as bigint)) AS (CAST(NULL AS STRING) <= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1330,7 +1642,7 @@ Project [NOT (cast(cast(null as string) as bigint) = cast(1 as bigint)) AS (NOT -- !query SELECT cast(1 as decimal(10, 0)) = '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1341,7 +1653,7 @@ Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DEC -- !query SELECT cast(1 as decimal(10, 0)) > '2' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) > cast(2 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 2)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(2 as double)) AS (CAST(1 AS DECIMAL(10,0)) > 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1352,7 +1664,7 @@ Project [(cast(1 as decimal(10,0)) > cast(2 as decimal(10,0))) AS (CAST(1 AS DEC -- !query SELECT cast(1 as decimal(10, 0)) >= '2' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) >= cast(2 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 2)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(2 as double)) AS (CAST(1 AS DECIMAL(10,0)) >= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1363,7 +1675,7 @@ Project [(cast(1 as decimal(10,0)) >= cast(2 as decimal(10,0))) AS (CAST(1 AS DE -- !query SELECT cast(1 as decimal(10, 0)) < '2' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) < cast(2 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 2)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(2 as double)) AS (CAST(1 AS DECIMAL(10,0)) < 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1374,7 +1686,7 @@ Project [(cast(1 as decimal(10,0)) < cast(2 as decimal(10,0))) AS (CAST(1 AS DEC -- !query SELECT cast(1 as decimal(10, 0)) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(2 as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 2))#x] +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(2 as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 2))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1385,7 +1697,7 @@ Project [NOT (cast(1 as decimal(10,0)) = cast(2 as decimal(10,0))) AS (NOT (CAST -- !query SELECT cast(1 as decimal(10, 0)) <= '2' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <= cast(2 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 2)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(2 as double)) AS (CAST(1 AS DECIMAL(10,0)) <= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1396,7 +1708,7 @@ Project [(cast(1 as decimal(10,0)) <= cast(2 as decimal(10,0))) AS (CAST(1 AS DE -- !query SELECT cast(1 as decimal(10, 0)) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(cast(null as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(cast(null as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1407,7 +1719,7 @@ Project [(cast(1 as decimal(10,0)) = cast(cast(null as string) as decimal(10,0)) -- !query SELECT cast(1 as decimal(10, 0)) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) > cast(cast(null as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(cast(null as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) > CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1418,7 +1730,7 @@ Project [(cast(1 as decimal(10,0)) > cast(cast(null as string) as decimal(10,0)) -- !query SELECT cast(1 as decimal(10, 0)) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) >= cast(cast(null as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(cast(null as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1429,7 +1741,7 @@ Project [(cast(1 as decimal(10,0)) >= cast(cast(null as string) as decimal(10,0) -- !query SELECT cast(1 as decimal(10, 0)) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) < cast(cast(null as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(cast(null as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) < CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1440,7 +1752,7 @@ Project [(cast(1 as decimal(10,0)) < cast(cast(null as string) as decimal(10,0)) -- !query SELECT cast(1 as decimal(10, 0)) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(cast(null as string) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING)))#x] +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(cast(null as string) as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1451,7 +1763,7 @@ Project [NOT (cast(1 as decimal(10,0)) = cast(cast(null as string) as decimal(10 -- !query SELECT cast(1 as decimal(10, 0)) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <= cast(cast(null as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(cast(null as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1462,7 +1774,7 @@ Project [(cast(1 as decimal(10,0)) <= cast(cast(null as string) as decimal(10,0) -- !query SELECT '1' = cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (1 = CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (1 = CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1473,7 +1785,7 @@ Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (1 = CAST(1 AS -- !query SELECT '2' > cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(2 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (2 > CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(2 as double) > cast(cast(1 as decimal(10,0)) as double)) AS (2 > CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1484,7 +1796,7 @@ Project [(cast(2 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (2 > CAST(1 AS -- !query SELECT '2' >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(2 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (2 >= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(2 as double) >= cast(cast(1 as decimal(10,0)) as double)) AS (2 >= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1495,7 +1807,7 @@ Project [(cast(2 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (2 >= CAST(1 -- !query SELECT '2' < cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(2 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (2 < CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(2 as double) < cast(cast(1 as decimal(10,0)) as double)) AS (2 < CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1506,7 +1818,7 @@ Project [(cast(2 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (2 < CAST(1 AS -- !query SELECT '2' <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(2 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (2 <= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(2 as double) <= cast(cast(1 as decimal(10,0)) as double)) AS (2 <= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1517,7 +1829,7 @@ Project [(cast(2 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (2 <= CAST(1 -- !query SELECT '2' <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [NOT (cast(2 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (2 = CAST(1 AS DECIMAL(10,0))))#x] +Project [NOT (cast(2 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (NOT (2 = CAST(1 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1528,7 +1840,7 @@ Project [NOT (cast(2 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (2 = -- !query SELECT cast(null as string) = cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(cast(null as string) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(cast(null as string) as double) = cast(cast(1 as decimal(10,0)) as double)) AS (CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1539,7 +1851,7 @@ Project [(cast(cast(null as string) as decimal(10,0)) = cast(1 as decimal(10,0)) -- !query SELECT cast(null as string) > cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(cast(null as string) as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(NULL AS STRING) > CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(cast(null as string) as double) > cast(cast(1 as decimal(10,0)) as double)) AS (CAST(NULL AS STRING) > CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1550,7 +1862,7 @@ Project [(cast(cast(null as string) as decimal(10,0)) > cast(1 as decimal(10,0)) -- !query SELECT cast(null as string) >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(cast(null as string) as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(NULL AS STRING) >= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(cast(null as string) as double) >= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(NULL AS STRING) >= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1561,7 +1873,7 @@ Project [(cast(cast(null as string) as decimal(10,0)) >= cast(1 as decimal(10,0) -- !query SELECT cast(null as string) < cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(cast(null as string) as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(NULL AS STRING) < CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(cast(null as string) as double) < cast(cast(1 as decimal(10,0)) as double)) AS (CAST(NULL AS STRING) < CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1572,7 +1884,7 @@ Project [(cast(cast(null as string) as decimal(10,0)) < cast(1 as decimal(10,0)) -- !query SELECT cast(null as string) <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(cast(null as string) as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(NULL AS STRING) <= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(cast(null as string) as double) <= cast(cast(1 as decimal(10,0)) as double)) AS (CAST(NULL AS STRING) <= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1583,7 +1895,7 @@ Project [(cast(cast(null as string) as decimal(10,0)) <= cast(1 as decimal(10,0) -- !query SELECT cast(null as string) <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0))))#x] +Project [NOT (cast(cast(null as string) as double) = cast(cast(1 as decimal(10,0)) as double)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1858,7 +2170,7 @@ Project [NOT (cast(cast(null as string) as double) = cast(1 as double)) AS (NOT -- !query SELECT cast(1 as float) = '1' FROM t -- !query analysis -Project [(cast(1 as float) = cast(1 as float)) AS (CAST(1 AS FLOAT) = 1)#x] +Project [(cast(cast(1 as float) as double) = cast(1 as double)) AS (CAST(1 AS FLOAT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1869,7 +2181,7 @@ Project [(cast(1 as float) = cast(1 as float)) AS (CAST(1 AS FLOAT) = 1)#x] -- !query SELECT cast(1 as float) > '2' FROM t -- !query analysis -Project [(cast(1 as float) > cast(2 as float)) AS (CAST(1 AS FLOAT) > 2)#x] +Project [(cast(cast(1 as float) as double) > cast(2 as double)) AS (CAST(1 AS FLOAT) > 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1880,7 +2192,7 @@ Project [(cast(1 as float) > cast(2 as float)) AS (CAST(1 AS FLOAT) > 2)#x] -- !query SELECT cast(1 as float) >= '2' FROM t -- !query analysis -Project [(cast(1 as float) >= cast(2 as float)) AS (CAST(1 AS FLOAT) >= 2)#x] +Project [(cast(cast(1 as float) as double) >= cast(2 as double)) AS (CAST(1 AS FLOAT) >= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1891,7 +2203,7 @@ Project [(cast(1 as float) >= cast(2 as float)) AS (CAST(1 AS FLOAT) >= 2)#x] -- !query SELECT cast(1 as float) < '2' FROM t -- !query analysis -Project [(cast(1 as float) < cast(2 as float)) AS (CAST(1 AS FLOAT) < 2)#x] +Project [(cast(cast(1 as float) as double) < cast(2 as double)) AS (CAST(1 AS FLOAT) < 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1902,7 +2214,7 @@ Project [(cast(1 as float) < cast(2 as float)) AS (CAST(1 AS FLOAT) < 2)#x] -- !query SELECT cast(1 as float) <= '2' FROM t -- !query analysis -Project [(cast(1 as float) <= cast(2 as float)) AS (CAST(1 AS FLOAT) <= 2)#x] +Project [(cast(cast(1 as float) as double) <= cast(2 as double)) AS (CAST(1 AS FLOAT) <= 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1913,7 +2225,7 @@ Project [(cast(1 as float) <= cast(2 as float)) AS (CAST(1 AS FLOAT) <= 2)#x] -- !query SELECT cast(1 as float) <> '2' FROM t -- !query analysis -Project [NOT (cast(1 as float) = cast(2 as float)) AS (NOT (CAST(1 AS FLOAT) = 2))#x] +Project [NOT (cast(cast(1 as float) as double) = cast(2 as double)) AS (NOT (CAST(1 AS FLOAT) = 2))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1924,7 +2236,7 @@ Project [NOT (cast(1 as float) = cast(2 as float)) AS (NOT (CAST(1 AS FLOAT) = 2 -- !query SELECT cast(1 as float) = cast(null as string) FROM t -- !query analysis -Project [(cast(1 as float) = cast(cast(null as string) as float)) AS (CAST(1 AS FLOAT) = CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as float) as double) = cast(cast(null as string) as double)) AS (CAST(1 AS FLOAT) = CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1935,7 +2247,7 @@ Project [(cast(1 as float) = cast(cast(null as string) as float)) AS (CAST(1 AS -- !query SELECT cast(1 as float) > cast(null as string) FROM t -- !query analysis -Project [(cast(1 as float) > cast(cast(null as string) as float)) AS (CAST(1 AS FLOAT) > CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as float) as double) > cast(cast(null as string) as double)) AS (CAST(1 AS FLOAT) > CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1946,7 +2258,7 @@ Project [(cast(1 as float) > cast(cast(null as string) as float)) AS (CAST(1 AS -- !query SELECT cast(1 as float) >= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as float) >= cast(cast(null as string) as float)) AS (CAST(1 AS FLOAT) >= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as float) as double) >= cast(cast(null as string) as double)) AS (CAST(1 AS FLOAT) >= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1957,7 +2269,7 @@ Project [(cast(1 as float) >= cast(cast(null as string) as float)) AS (CAST(1 AS -- !query SELECT cast(1 as float) < cast(null as string) FROM t -- !query analysis -Project [(cast(1 as float) < cast(cast(null as string) as float)) AS (CAST(1 AS FLOAT) < CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as float) as double) < cast(cast(null as string) as double)) AS (CAST(1 AS FLOAT) < CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1968,7 +2280,7 @@ Project [(cast(1 as float) < cast(cast(null as string) as float)) AS (CAST(1 AS -- !query SELECT cast(1 as float) <= cast(null as string) FROM t -- !query analysis -Project [(cast(1 as float) <= cast(cast(null as string) as float)) AS (CAST(1 AS FLOAT) <= CAST(NULL AS STRING))#x] +Project [(cast(cast(1 as float) as double) <= cast(cast(null as string) as double)) AS (CAST(1 AS FLOAT) <= CAST(NULL AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1979,7 +2291,7 @@ Project [(cast(1 as float) <= cast(cast(null as string) as float)) AS (CAST(1 AS -- !query SELECT cast(1 as float) <> cast(null as string) FROM t -- !query analysis -Project [NOT (cast(1 as float) = cast(cast(null as string) as float)) AS (NOT (CAST(1 AS FLOAT) = CAST(NULL AS STRING)))#x] +Project [NOT (cast(cast(1 as float) as double) = cast(cast(null as string) as double)) AS (NOT (CAST(1 AS FLOAT) = CAST(NULL AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1990,7 +2302,7 @@ Project [NOT (cast(1 as float) = cast(cast(null as string) as float)) AS (NOT (C -- !query SELECT '1' = cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) = cast(1 as float)) AS (1 = CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) = cast(cast(1 as float) as double)) AS (1 = CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2001,7 +2313,7 @@ Project [(cast(1 as float) = cast(1 as float)) AS (1 = CAST(1 AS FLOAT))#x] -- !query SELECT '2' > cast(1 as float) FROM t -- !query analysis -Project [(cast(2 as float) > cast(1 as float)) AS (2 > CAST(1 AS FLOAT))#x] +Project [(cast(2 as double) > cast(cast(1 as float) as double)) AS (2 > CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2012,7 +2324,7 @@ Project [(cast(2 as float) > cast(1 as float)) AS (2 > CAST(1 AS FLOAT))#x] -- !query SELECT '2' >= cast(1 as float) FROM t -- !query analysis -Project [(cast(2 as float) >= cast(1 as float)) AS (2 >= CAST(1 AS FLOAT))#x] +Project [(cast(2 as double) >= cast(cast(1 as float) as double)) AS (2 >= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2023,7 +2335,7 @@ Project [(cast(2 as float) >= cast(1 as float)) AS (2 >= CAST(1 AS FLOAT))#x] -- !query SELECT '2' < cast(1 as float) FROM t -- !query analysis -Project [(cast(2 as float) < cast(1 as float)) AS (2 < CAST(1 AS FLOAT))#x] +Project [(cast(2 as double) < cast(cast(1 as float) as double)) AS (2 < CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2034,7 +2346,7 @@ Project [(cast(2 as float) < cast(1 as float)) AS (2 < CAST(1 AS FLOAT))#x] -- !query SELECT '2' <= cast(1 as float) FROM t -- !query analysis -Project [(cast(2 as float) <= cast(1 as float)) AS (2 <= CAST(1 AS FLOAT))#x] +Project [(cast(2 as double) <= cast(cast(1 as float) as double)) AS (2 <= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2045,7 +2357,7 @@ Project [(cast(2 as float) <= cast(1 as float)) AS (2 <= CAST(1 AS FLOAT))#x] -- !query SELECT '2' <> cast(1 as float) FROM t -- !query analysis -Project [NOT (cast(2 as float) = cast(1 as float)) AS (NOT (2 = CAST(1 AS FLOAT)))#x] +Project [NOT (cast(2 as double) = cast(cast(1 as float) as double)) AS (NOT (2 = CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2056,7 +2368,7 @@ Project [NOT (cast(2 as float) = cast(1 as float)) AS (NOT (2 = CAST(1 AS FLOAT) -- !query SELECT cast(null as string) = cast(1 as float) FROM t -- !query analysis -Project [(cast(cast(null as string) as float) = cast(1 as float)) AS (CAST(NULL AS STRING) = CAST(1 AS FLOAT))#x] +Project [(cast(cast(null as string) as double) = cast(cast(1 as float) as double)) AS (CAST(NULL AS STRING) = CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2067,7 +2379,7 @@ Project [(cast(cast(null as string) as float) = cast(1 as float)) AS (CAST(NULL -- !query SELECT cast(null as string) > cast(1 as float) FROM t -- !query analysis -Project [(cast(cast(null as string) as float) > cast(1 as float)) AS (CAST(NULL AS STRING) > CAST(1 AS FLOAT))#x] +Project [(cast(cast(null as string) as double) > cast(cast(1 as float) as double)) AS (CAST(NULL AS STRING) > CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2078,7 +2390,7 @@ Project [(cast(cast(null as string) as float) > cast(1 as float)) AS (CAST(NULL -- !query SELECT cast(null as string) >= cast(1 as float) FROM t -- !query analysis -Project [(cast(cast(null as string) as float) >= cast(1 as float)) AS (CAST(NULL AS STRING) >= CAST(1 AS FLOAT))#x] +Project [(cast(cast(null as string) as double) >= cast(cast(1 as float) as double)) AS (CAST(NULL AS STRING) >= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2089,7 +2401,7 @@ Project [(cast(cast(null as string) as float) >= cast(1 as float)) AS (CAST(NULL -- !query SELECT cast(null as string) < cast(1 as float) FROM t -- !query analysis -Project [(cast(cast(null as string) as float) < cast(1 as float)) AS (CAST(NULL AS STRING) < CAST(1 AS FLOAT))#x] +Project [(cast(cast(null as string) as double) < cast(cast(1 as float) as double)) AS (CAST(NULL AS STRING) < CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2100,7 +2412,7 @@ Project [(cast(cast(null as string) as float) < cast(1 as float)) AS (CAST(NULL -- !query SELECT cast(null as string) <= cast(1 as float) FROM t -- !query analysis -Project [(cast(cast(null as string) as float) <= cast(1 as float)) AS (CAST(NULL AS STRING) <= CAST(1 AS FLOAT))#x] +Project [(cast(cast(null as string) as double) <= cast(cast(1 as float) as double)) AS (CAST(NULL AS STRING) <= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2111,7 +2423,7 @@ Project [(cast(cast(null as string) as float) <= cast(1 as float)) AS (CAST(NULL -- !query SELECT cast(null as string) <> cast(1 as float) FROM t -- !query analysis -Project [NOT (cast(cast(null as string) as float) = cast(1 as float)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS FLOAT)))#x] +Project [NOT (cast(cast(null as string) as double) = cast(cast(1 as float) as double)) AS (NOT (CAST(NULL AS STRING) = CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out index de6c0b72c1c79..4fe85374f4b1b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/booleanEquality.sql.out @@ -10,78 +10,155 @@ CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, UNSUPPORTED, true -- !query SELECT true = cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(true as tinyint) = cast(1 as tinyint)) AS (true = CAST(1 AS TINYINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true = CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "true = cast(1 as tinyint)" + } ] +} -- !query SELECT true = cast(1 as smallint) FROM t -- !query analysis -Project [(cast(true as smallint) = cast(1 as smallint)) AS (true = CAST(1 AS SMALLINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true = CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true = cast(1 as smallint)" + } ] +} -- !query SELECT true = cast(1 as int) FROM t -- !query analysis -Project [(cast(true as int) = cast(1 as int)) AS (true = CAST(1 AS INT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true = CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "true = cast(1 as int)" + } ] +} -- !query SELECT true = cast(1 as bigint) FROM t -- !query analysis -Project [(cast(true as bigint) = cast(1 as bigint)) AS (true = CAST(1 AS BIGINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true = CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as bigint)" + } ] +} -- !query SELECT true = cast(1 as float) FROM t -- !query analysis -Project [(cast(true as float) = cast(1 as float)) AS (true = CAST(1 AS FLOAT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true = CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true = cast(1 as float)" + } ] +} -- !query SELECT true = cast(1 as double) FROM t -- !query analysis -Project [(cast(true as double) = cast(1 as double)) AS (true = CAST(1 AS DOUBLE))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true = CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as double)" + } ] +} -- !query SELECT true = cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(true as decimal(10,0)) = cast(1 as decimal(10,0))) AS (true = CAST(1 AS DECIMAL(10,0)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "true = cast(1 as decimal(10, 0))" + } ] +} -- !query @@ -175,126 +252,71 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT true <=> cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(true as tinyint) <=> cast(1 as tinyint)) AS (true <=> CAST(1 AS TINYINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "true <=> cast(1 as tinyint)" + } ] +} -- !query SELECT true <=> cast(1 as smallint) FROM t -- !query analysis -Project [(cast(true as smallint) <=> cast(1 as smallint)) AS (true <=> CAST(1 AS SMALLINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as int) FROM t --- !query analysis -Project [(cast(true as int) <=> cast(1 as int)) AS (true <=> CAST(1 AS INT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as bigint) FROM t --- !query analysis -Project [(cast(true as bigint) <=> cast(1 as bigint)) AS (true <=> CAST(1 AS BIGINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as float) FROM t --- !query analysis -Project [(cast(true as float) <=> cast(1 as float)) AS (true <=> CAST(1 AS FLOAT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as double) FROM t --- !query analysis -Project [(cast(true as double) <=> cast(1 as double)) AS (true <=> CAST(1 AS DOUBLE))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as decimal(10, 0)) FROM t --- !query analysis -Project [(cast(true as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (true <=> CAST(1 AS DECIMAL(10,0)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast(1 as string) FROM t --- !query analysis -Project [(true <=> cast(cast(1 as string) as boolean)) AS (true <=> CAST(1 AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT true <=> cast('1' as binary) FROM t --- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"", - "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"" + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS SMALLINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, "stopIndex" : 35, - "fragment" : "true <=> cast('1' as binary)" + "fragment" : "true <=> cast(1 as smallint)" } ] } -- !query -SELECT true <=> cast(1 as boolean) FROM t +SELECT true <=> cast(1 as int) FROM t -- !query analysis -Project [(true <=> cast(1 as boolean)) AS (true <=> CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true <=> cast(1 as int)" + } ] +} -- !query -SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +SELECT true <=> cast(1 as bigint) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { @@ -302,21 +324,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"", - "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BIGINT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 58, - "fragment" : "true <=> cast('2017-12-11 09:30:00.0' as timestamp)" + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as bigint)" } ] } -- !query -SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t +SELECT true <=> cast(1 as float) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { @@ -324,23 +346,67 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"", - "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS FLOAT))\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 51, - "fragment" : "true <=> cast('2017-12-11 09:30:00' as date)" + "stopIndex" : 32, + "fragment" : "true <=> cast(1 as float)" } ] } -- !query -SELECT cast(1 as tinyint) = true FROM t +SELECT true <=> cast(1 as double) FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(true as tinyint)) AS (CAST(1 AS TINYINT) = true)#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as double)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as decimal(10, 0)) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "true <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT true <=> cast(1 as string) FROM t +-- !query analysis +Project [(true <=> cast(cast(1 as string) as boolean)) AS (true <=> CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -349,9 +415,31 @@ Project [(cast(1 as tinyint) = cast(true as tinyint)) AS (CAST(1 AS TINYINT) = t -- !query -SELECT cast(1 as smallint) = true FROM t +SELECT true <=> cast('1' as binary) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "true <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(true as smallint)) AS (CAST(1 AS SMALLINT) = true)#x] +Project [(true <=> cast(1 as boolean)) AS (true <=> CAST(1 AS BOOLEAN))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -359,59 +447,202 @@ Project [(cast(1 as smallint) = cast(true as smallint)) AS (CAST(1 AS SMALLINT) +- OneRowRelation +-- !query +SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "true <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "true <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = true FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as tinyint) = true" + } ] +} + + +-- !query +SELECT cast(1 as smallint) = true FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as smallint) = true" + } ] +} + + -- !query SELECT cast(1 as int) = true FROM t -- !query analysis -Project [(cast(1 as int) = cast(true as int)) AS (CAST(1 AS INT) = true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast(1 as int) = true" + } ] +} -- !query SELECT cast(1 as bigint) = true FROM t -- !query analysis -Project [(cast(1 as bigint) = cast(true as bigint)) AS (CAST(1 AS BIGINT) = true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as bigint) = true" + } ] +} -- !query SELECT cast(1 as float) = true FROM t -- !query analysis -Project [(cast(1 as float) = cast(true as float)) AS (CAST(1 AS FLOAT) = true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as float) = true" + } ] +} -- !query SELECT cast(1 as double) = true FROM t -- !query analysis -Project [(cast(1 as double) = cast(true as double)) AS (CAST(1 AS DOUBLE) = true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as double) = true" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) = true FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(true as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast(1 as decimal(10, 0)) = true" + } ] +} -- !query @@ -505,78 +736,155 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) <=> true FROM t -- !query analysis -Project [(cast(1 as tinyint) <=> cast(true as tinyint)) AS (CAST(1 AS TINYINT) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(1 as tinyint) <=> true" + } ] +} -- !query SELECT cast(1 as smallint) <=> true FROM t -- !query analysis -Project [(cast(1 as smallint) <=> cast(true as smallint)) AS (CAST(1 AS SMALLINT) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(1 as smallint) <=> true" + } ] +} -- !query SELECT cast(1 as int) <=> true FROM t -- !query analysis -Project [(cast(1 as int) <=> cast(true as int)) AS (CAST(1 AS INT) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as int) <=> true" + } ] +} -- !query SELECT cast(1 as bigint) <=> true FROM t -- !query analysis -Project [(cast(1 as bigint) <=> cast(true as bigint)) AS (CAST(1 AS BIGINT) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as bigint) <=> true" + } ] +} -- !query SELECT cast(1 as float) <=> true FROM t -- !query analysis -Project [(cast(1 as float) <=> cast(true as float)) AS (CAST(1 AS FLOAT) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as float) <=> true" + } ] +} -- !query SELECT cast(1 as double) <=> true FROM t -- !query analysis -Project [(cast(1 as double) <=> cast(true as double)) AS (CAST(1 AS DOUBLE) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as double) <=> true" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <=> true FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <=> cast(true as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> true)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as decimal(10, 0)) <=> true" + } ] +} -- !query @@ -670,78 +978,155 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT false = cast(0 as tinyint) FROM t -- !query analysis -Project [(cast(false as tinyint) = cast(0 as tinyint)) AS (false = CAST(0 AS TINYINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false = CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false = cast(0 as tinyint)" + } ] +} -- !query SELECT false = cast(0 as smallint) FROM t -- !query analysis -Project [(cast(false as smallint) = cast(0 as smallint)) AS (false = CAST(0 AS SMALLINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false = CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false = cast(0 as smallint)" + } ] +} -- !query SELECT false = cast(0 as int) FROM t -- !query analysis -Project [(cast(false as int) = cast(0 as int)) AS (false = CAST(0 AS INT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false = CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "false = cast(0 as int)" + } ] +} -- !query SELECT false = cast(0 as bigint) FROM t -- !query analysis -Project [(cast(false as bigint) = cast(0 as bigint)) AS (false = CAST(0 AS BIGINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false = CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as bigint)" + } ] +} -- !query SELECT false = cast(0 as float) FROM t -- !query analysis -Project [(cast(false as float) = cast(0 as float)) AS (false = CAST(0 AS FLOAT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false = CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false = cast(0 as float)" + } ] +} -- !query SELECT false = cast(0 as double) FROM t -- !query analysis -Project [(cast(false as double) = cast(0 as double)) AS (false = CAST(0 AS DOUBLE))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false = CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as double)" + } ] +} -- !query SELECT false = cast(0 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(false as decimal(10,0)) = cast(0 as decimal(10,0))) AS (false = CAST(0 AS DECIMAL(10,0)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false = CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "false = cast(0 as decimal(10, 0))" + } ] +} -- !query @@ -835,73 +1220,161 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT false <=> cast(0 as tinyint) FROM t -- !query analysis -Project [(cast(false as tinyint) <=> cast(0 as tinyint)) AS (false <=> CAST(0 AS TINYINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "false <=> cast(0 as tinyint)" + } ] +} -- !query SELECT false <=> cast(0 as smallint) FROM t -- !query analysis -Project [(cast(false as smallint) <=> cast(0 as smallint)) AS (false <=> CAST(0 AS SMALLINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast(0 as smallint)" + } ] +} -- !query SELECT false <=> cast(0 as int) FROM t -- !query analysis -Project [(cast(false as int) <=> cast(0 as int)) AS (false <=> CAST(0 AS INT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false <=> cast(0 as int)" + } ] +} -- !query SELECT false <=> cast(0 as bigint) FROM t -- !query analysis -Project [(cast(false as bigint) <=> cast(0 as bigint)) AS (false <=> CAST(0 AS BIGINT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as bigint)" + } ] +} -- !query SELECT false <=> cast(0 as float) FROM t -- !query analysis -Project [(cast(false as float) <=> cast(0 as float)) AS (false <=> CAST(0 AS FLOAT))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false <=> cast(0 as float)" + } ] +} -- !query SELECT false <=> cast(0 as double) FROM t -- !query analysis -Project [(cast(false as double) <=> cast(0 as double)) AS (false <=> CAST(0 AS DOUBLE))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as double)" + } ] +} -- !query SELECT false <=> cast(0 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(false as decimal(10,0)) <=> cast(0 as decimal(10,0))) AS (false <=> CAST(0 AS DECIMAL(10,0)))#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "false <=> cast(0 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT false <=> cast(0 as string) FROM t +-- !query analysis +Project [(false <=> cast(cast(0 as string) as boolean)) AS (false <=> CAST(0 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -910,9 +1383,31 @@ Project [(cast(false as decimal(10,0)) <=> cast(0 as decimal(10,0))) AS (false < -- !query -SELECT false <=> cast(0 as string) FROM t +SELECT false <=> cast('0' as binary) FROM t -- !query analysis -Project [(false <=> cast(cast(0 as string) as boolean)) AS (false <=> CAST(0 AS STRING))#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast('0' as binary)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as boolean) FROM t +-- !query analysis +Project [(false <=> cast(0 as boolean)) AS (false <=> CAST(0 AS BOOLEAN))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -921,157 +1416,201 @@ Project [(false <=> cast(cast(0 as string) as boolean)) AS (false <=> CAST(0 AS -- !query -SELECT false <=> cast('0' as binary) FROM t +SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "false <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "false <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(0 as tinyint) = false FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as tinyint) = false" + } ] +} + + +-- !query +SELECT cast(0 as smallint) = false FROM t +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as smallint) = false" + } ] +} + + +-- !query +SELECT cast(0 as int) = false FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"BINARY\"", - "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"" + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 36, - "fragment" : "false <=> cast('0' as binary)" + "stopIndex" : 29, + "fragment" : "cast(0 as int) = false" } ] } -- !query -SELECT false <=> cast(0 as boolean) FROM t --- !query analysis -Project [(false <=> cast(0 as boolean)) AS (false <=> CAST(0 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +SELECT cast(0 as bigint) = false FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"TIMESTAMP\"", - "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 59, - "fragment" : "false <=> cast('2017-12-11 09:30:00.0' as timestamp)" + "stopIndex" : 32, + "fragment" : "cast(0 as bigint) = false" } ] } -- !query -SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t +SELECT cast(0 as float) = false FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DATE\"", - "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) = false)\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 52, - "fragment" : "false <=> cast('2017-12-11 09:30:00' as date)" + "stopIndex" : 31, + "fragment" : "cast(0 as float) = false" } ] } --- !query -SELECT cast(0 as tinyint) = false FROM t --- !query analysis -Project [(cast(0 as tinyint) = cast(false as tinyint)) AS (CAST(0 AS TINYINT) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT cast(0 as smallint) = false FROM t --- !query analysis -Project [(cast(0 as smallint) = cast(false as smallint)) AS (CAST(0 AS SMALLINT) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT cast(0 as int) = false FROM t --- !query analysis -Project [(cast(0 as int) = cast(false as int)) AS (CAST(0 AS INT) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT cast(0 as bigint) = false FROM t --- !query analysis -Project [(cast(0 as bigint) = cast(false as bigint)) AS (CAST(0 AS BIGINT) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - --- !query -SELECT cast(0 as float) = false FROM t --- !query analysis -Project [(cast(0 as float) = cast(false as float)) AS (CAST(0 AS FLOAT) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation - - -- !query SELECT cast(0 as double) = false FROM t -- !query analysis -Project [(cast(0 as double) = cast(false as double)) AS (CAST(0 AS DOUBLE) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(0 as double) = false" + } ] +} -- !query SELECT cast(0 as decimal(10, 0)) = false FROM t -- !query analysis -Project [(cast(0 as decimal(10,0)) = cast(false as decimal(10,0))) AS (CAST(0 AS DECIMAL(10,0)) = false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(0 as decimal(10, 0)) = false" + } ] +} -- !query @@ -1165,78 +1704,155 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(0 as tinyint) <=> false FROM t -- !query analysis -Project [(cast(0 as tinyint) <=> cast(false as tinyint)) AS (CAST(0 AS TINYINT) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(0 as tinyint) <=> false" + } ] +} -- !query SELECT cast(0 as smallint) <=> false FROM t -- !query analysis -Project [(cast(0 as smallint) <=> cast(false as smallint)) AS (CAST(0 AS SMALLINT) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast(0 as smallint) <=> false" + } ] +} -- !query SELECT cast(0 as int) <=> false FROM t -- !query analysis -Project [(cast(0 as int) <=> cast(false as int)) AS (CAST(0 AS INT) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(0 as int) <=> false" + } ] +} -- !query SELECT cast(0 as bigint) <=> false FROM t -- !query analysis -Project [(cast(0 as bigint) <=> cast(false as bigint)) AS (CAST(0 AS BIGINT) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as bigint) <=> false" + } ] +} -- !query SELECT cast(0 as float) <=> false FROM t -- !query analysis -Project [(cast(0 as float) <=> cast(false as float)) AS (CAST(0 AS FLOAT) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as float) <=> false" + } ] +} -- !query SELECT cast(0 as double) <=> false FROM t -- !query analysis -Project [(cast(0 as double) <=> cast(false as double)) AS (CAST(0 AS DOUBLE) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as double) <=> false" + } ] +} -- !query SELECT cast(0 as decimal(10, 0)) <=> false FROM t -- !query analysis -Project [(cast(0 as decimal(10,0)) <=> cast(false as decimal(10,0))) AS (CAST(0 AS DECIMAL(10,0)) <=> false)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(0 as decimal(10, 0)) <=> false" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out index 4124fc57996cd..45ce3bdd96d90 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/caseWhenCoercion.sql.out @@ -54,7 +54,7 @@ Project [CASE WHEN true THEN cast(cast(1 as tinyint) as bigint) ELSE cast(2 as b -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as tinyint) as float) ELSE cast(2 as float) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS FLOAT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS FLOAT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -87,7 +87,7 @@ Project [CASE WHEN true THEN cast(cast(1 as tinyint) as decimal(10,0)) ELSE cast -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as tinyint) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(cast(1 as tinyint) as bigint) ELSE cast(cast(2 as string) as bigint) END AS CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS STRING) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -230,7 +230,7 @@ Project [CASE WHEN true THEN cast(cast(1 as smallint) as bigint) ELSE cast(2 as -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as float) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as smallint) as float) ELSE cast(2 as float) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS FLOAT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as smallint) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS FLOAT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -263,7 +263,7 @@ Project [CASE WHEN true THEN cast(cast(1 as smallint) as decimal(10,0)) ELSE cas -- !query SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as smallint) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(cast(1 as smallint) as bigint) ELSE cast(cast(2 as string) as bigint) END AS CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS STRING) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -406,7 +406,7 @@ Project [CASE WHEN true THEN cast(cast(1 as int) as bigint) ELSE cast(2 as bigin -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as float) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as int) as float) ELSE cast(2 as float) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS FLOAT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as int) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS FLOAT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -439,7 +439,7 @@ Project [CASE WHEN true THEN cast(cast(1 as int) as decimal(10,0)) ELSE cast(2 a -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as int) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(cast(1 as int) as bigint) ELSE cast(cast(2 as string) as bigint) END AS CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS STRING) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -582,7 +582,7 @@ Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as bigint) END AS CAS -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as float) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as bigint) as float) ELSE cast(2 as float) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS FLOAT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as bigint) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS FLOAT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -615,7 +615,7 @@ Project [CASE WHEN true THEN cast(cast(1 as bigint) as decimal(20,0)) ELSE cast( -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as bigint) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(1 as bigint) ELSE cast(cast(2 as string) as bigint) END AS CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS STRING) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -714,7 +714,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as tinyint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as tinyint) as float) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS TINYINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as tinyint) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS TINYINT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -725,7 +725,7 @@ Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as tinyint) as fl -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as smallint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as smallint) as float) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS SMALLINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as smallint) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS SMALLINT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -736,7 +736,7 @@ Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as smallint) as f -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as int) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as int) as float) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS INT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as int) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS INT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -747,7 +747,7 @@ Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as int) as float) -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as bigint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as float) ELSE cast(cast(2 as bigint) as float) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BIGINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as bigint) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BIGINT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -791,7 +791,7 @@ Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 a -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as float) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(cast(1 as float) as double) ELSE cast(cast(2 as string) as double) END AS CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS STRING) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -967,7 +967,7 @@ Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as decimal(10,0) -- !query SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as double) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(1 as double) ELSE cast(cast(2 as string) as double) END AS CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS STRING) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1143,7 +1143,7 @@ Project [CASE WHEN true THEN cast(1 as decimal(10,0)) ELSE cast(2 as decimal(10, -- !query SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(cast(1 as decimal(10,0)) as double) ELSE cast(cast(2 as string) as double) END AS CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS STRING) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1242,7 +1242,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as tinyint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as tinyint) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS TINYINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as bigint) ELSE cast(cast(2 as tinyint) as bigint) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS TINYINT) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1253,7 +1253,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as tinyint) as s -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as smallint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as smallint) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS SMALLINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as bigint) ELSE cast(cast(2 as smallint) as bigint) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS SMALLINT) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1264,7 +1264,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as smallint) as -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as int) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as int) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS INT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as bigint) ELSE cast(cast(2 as int) as bigint) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS INT) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1275,7 +1275,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as int) as strin -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as bigint) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as bigint) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BIGINT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as bigint) ELSE cast(2 as bigint) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BIGINT) END#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1286,7 +1286,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as bigint) as st -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as float) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as float) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS FLOAT) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as double) ELSE cast(cast(2 as float) as double) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS FLOAT) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1297,7 +1297,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as float) as str -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as double) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as double) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS DOUBLE) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as double) ELSE cast(2 as double) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS DOUBLE) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1308,7 +1308,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as double) as st -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2 as decimal(10,0)) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS DECIMAL(10,0)) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as double) ELSE cast(cast(2 as decimal(10,0)) as double) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS DECIMAL(10,0)) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1330,51 +1330,29 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END AS CAS -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BINARY) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 73, - "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END" - } ] -} +Project [CASE WHEN true THEN cast(cast(1 as string) as binary) ELSE cast(2 as binary) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BINARY) END#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 72, - "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END" - } ] -} +Project [CASE WHEN true THEN cast(cast(1 as string) as boolean) ELSE cast(2 as boolean) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2017-12-11 09:30:00.0 as timestamp) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as timestamp) ELSE cast(2017-12-11 09:30:00.0 as timestamp) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1385,7 +1363,7 @@ Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2017-12-11 09:30:0 -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(1 as string) ELSE cast(cast(2017-12-11 09:30:00 as date) as string) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2017-12-11 09:30:00 AS DATE) END#x] +Project [CASE WHEN true THEN cast(cast(1 as string) as date) ELSE cast(2017-12-11 09:30:00 as date) END AS CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2017-12-11 09:30:00 AS DATE) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1550,23 +1528,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS STRING) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 73, - "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END" - } ] -} +Project [CASE WHEN true THEN cast(1 as binary) ELSE cast(cast(2 as string) as binary) END AS CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS STRING) END#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1803,23 +1770,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 72, - "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END" - } ] -} +Project [CASE WHEN true THEN cast(1 as boolean) ELSE cast(cast(2 as string) as boolean) END AS CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2056,7 +2012,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(2017-12-12 09:30:00.0 as timestamp) ELSE cast(cast(2 as string) as timestamp) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS STRING) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2287,7 +2243,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as string) END FROM t -- !query analysis -Project [CASE WHEN true THEN cast(cast(2017-12-12 09:30:00 as date) as string) ELSE cast(2 as string) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS STRING) END#x] +Project [CASE WHEN true THEN cast(2017-12-12 09:30:00 as date) ELSE cast(cast(2 as string) as date) END AS CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS STRING) END#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out index 62e3a87473263..6f3bc9ccb66f3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/concat.sql.out @@ -271,7 +271,7 @@ SELECT (string_array1 || int_array2) sti_array FROM various_arrays -- !query analysis -Project [concat(cast(tinyint_array1#x as array), smallint_array2#x) AS ts_array#x, concat(cast(smallint_array1#x as array), int_array2#x) AS si_array#x, concat(cast(int_array1#x as array), bigint_array2#x) AS ib_array#x, concat(cast(bigint_array1#x as array), cast(decimal_array2#x as array)) AS bd_array#x, concat(cast(decimal_array1#x as array), double_array2#x) AS dd_array#x, concat(double_array1#x, cast(float_array2#x as array)) AS df_array#x, concat(string_array1#x, cast(data_array2#x as array)) AS std_array#x, concat(cast(timestamp_array1#x as array), string_array2#x) AS tst_array#x, concat(string_array1#x, cast(int_array2#x as array)) AS sti_array#x] +Project [concat(cast(tinyint_array1#x as array), smallint_array2#x) AS ts_array#x, concat(cast(smallint_array1#x as array), int_array2#x) AS si_array#x, concat(cast(int_array1#x as array), bigint_array2#x) AS ib_array#x, concat(cast(bigint_array1#x as array), cast(decimal_array2#x as array)) AS bd_array#x, concat(cast(decimal_array1#x as array), double_array2#x) AS dd_array#x, concat(double_array1#x, cast(float_array2#x as array)) AS df_array#x, concat(cast(string_array1#x as array), data_array2#x) AS std_array#x, concat(timestamp_array1#x, cast(string_array2#x as array)) AS tst_array#x, concat(cast(string_array1#x as array), cast(int_array2#x as array)) AS sti_array#x] +- SubqueryAlias various_arrays +- View (`various_arrays`, [boolean_array1#x, boolean_array2#x, tinyint_array1#x, tinyint_array2#x, smallint_array1#x, smallint_array2#x, int_array1#x, int_array2#x, bigint_array1#x, bigint_array2#x, decimal_array1#x, decimal_array2#x, double_array1#x, double_array2#x, float_array1#x, float_array2#x, date_array1#x, data_array2#x, timestamp_array1#x, timestamp_array2#x, string_array1#x, string_array2#x, array_array1#x, array_array2#x, ... 4 more fields]) +- Project [cast(boolean_array1#x as array) AS boolean_array1#x, cast(boolean_array2#x as array) AS boolean_array2#x, cast(tinyint_array1#x as array) AS tinyint_array1#x, cast(tinyint_array2#x as array) AS tinyint_array2#x, cast(smallint_array1#x as array) AS smallint_array1#x, cast(smallint_array2#x as array) AS smallint_array2#x, cast(int_array1#x as array) AS int_array1#x, cast(int_array2#x as array) AS int_array2#x, cast(bigint_array1#x as array) AS bigint_array1#x, cast(bigint_array2#x as array) AS bigint_array2#x, cast(decimal_array1#x as array) AS decimal_array1#x, cast(decimal_array2#x as array) AS decimal_array2#x, cast(double_array1#x as array) AS double_array1#x, cast(double_array2#x as array) AS double_array2#x, cast(float_array1#x as array) AS float_array1#x, cast(float_array2#x as array) AS float_array2#x, cast(date_array1#x as array) AS date_array1#x, cast(data_array2#x as array) AS data_array2#x, cast(timestamp_array1#x as array) AS timestamp_array1#x, cast(timestamp_array2#x as array) AS timestamp_array2#x, cast(string_array1#x as array) AS string_array1#x, cast(string_array2#x as array) AS string_array2#x, cast(array_array1#x as array>) AS array_array1#x, cast(array_array2#x as array>) AS array_array2#x, ... 4 more fields] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out index 093297f03edb7..4458e15e53cf7 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/decimalPrecision.sql.out @@ -8594,7 +8594,7 @@ Project [(cast(1 as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decimal(20 -- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) = cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) = cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -8605,7 +8605,7 @@ Project [(cast(1 as decimal(3,0)) = cast(cast(1 as string) as decimal(3,0))) AS -- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) = cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) = cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -8616,7 +8616,7 @@ Project [(cast(1 as decimal(5,0)) = cast(cast(1 as string) as decimal(5,0))) AS -- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -8627,7 +8627,7 @@ Project [(cast(1 as decimal(10,0)) = cast(cast(1 as string) as decimal(10,0))) A -- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) = cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -8726,45 +8726,89 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) = cast(cast(1 as boolean) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) = cast(cast(1 as boolean) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(cast(1 as boolean) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) = cast(cast(1 as boolean) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) = cast(1 as boolean)" + } ] +} -- !query @@ -9826,7 +9870,7 @@ Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as decimal(10,0)) as decimal( -- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) <=> cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) <=> cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -9837,7 +9881,7 @@ Project [(cast(1 as decimal(3,0)) <=> cast(cast(1 as string) as decimal(3,0))) A -- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) <=> cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) <=> cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -9848,7 +9892,7 @@ Project [(cast(1 as decimal(5,0)) <=> cast(cast(1 as string) as decimal(5,0))) A -- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <=> cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -9859,7 +9903,7 @@ Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as string) as decimal(10,0))) -- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) <=> cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -9958,45 +10002,89 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) <=> cast(cast(1 as boolean) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) <=> cast(cast(1 as boolean) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <=> cast(cast(1 as boolean) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) <=> cast(cast(1 as boolean) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BOOLEAN))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast(1 as boolean)" + } ] +} -- !query @@ -11058,7 +11146,7 @@ Project [(cast(1 as decimal(20,0)) < cast(cast(1 as decimal(10,0)) as decimal(20 -- !query SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) < cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) < CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) < cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) < CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -11069,7 +11157,7 @@ Project [(cast(1 as decimal(3,0)) < cast(cast(1 as string) as decimal(3,0))) AS -- !query SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) < cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) < CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) < cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) < CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -11080,7 +11168,7 @@ Project [(cast(1 as decimal(5,0)) < cast(cast(1 as string) as decimal(5,0))) AS -- !query SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) < cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) < CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -11091,7 +11179,7 @@ Project [(cast(1 as decimal(10,0)) < cast(cast(1 as string) as decimal(10,0))) A -- !query SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) < cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) < CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) < cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) < CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -12334,7 +12422,7 @@ Project [(cast(1 as decimal(20,0)) <= cast(cast(1 as decimal(10,0)) as decimal(2 -- !query SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) <= cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) <= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -12345,7 +12433,7 @@ Project [(cast(1 as decimal(3,0)) <= cast(cast(1 as string) as decimal(3,0))) AS -- !query SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) <= cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) <= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -12356,7 +12444,7 @@ Project [(cast(1 as decimal(5,0)) <= cast(cast(1 as string) as decimal(5,0))) AS -- !query SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <= cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -12367,7 +12455,7 @@ Project [(cast(1 as decimal(10,0)) <= cast(cast(1 as string) as decimal(10,0))) -- !query SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) <= cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) <= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -13610,7 +13698,7 @@ Project [(cast(1 as decimal(20,0)) > cast(cast(1 as decimal(10,0)) as decimal(20 -- !query SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) > cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) > CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) > cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) > CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -13621,7 +13709,7 @@ Project [(cast(1 as decimal(3,0)) > cast(cast(1 as string) as decimal(3,0))) AS -- !query SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) > cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) > CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) > cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) > CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -13632,7 +13720,7 @@ Project [(cast(1 as decimal(5,0)) > cast(cast(1 as string) as decimal(5,0))) AS -- !query SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) > cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) > CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -13643,7 +13731,7 @@ Project [(cast(1 as decimal(10,0)) > cast(cast(1 as string) as decimal(10,0))) A -- !query SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) > cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) > CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) > cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) > CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -14886,7 +14974,7 @@ Project [(cast(1 as decimal(20,0)) >= cast(cast(1 as decimal(10,0)) as decimal(2 -- !query SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(3,0)) >= cast(cast(1 as string) as decimal(3,0))) AS (CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(3,0)) as double) >= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -14897,7 +14985,7 @@ Project [(cast(1 as decimal(3,0)) >= cast(cast(1 as string) as decimal(3,0))) AS -- !query SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(5,0)) >= cast(cast(1 as string) as decimal(5,0))) AS (CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(5,0)) as double) >= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -14908,7 +14996,7 @@ Project [(cast(1 as decimal(5,0)) >= cast(cast(1 as string) as decimal(5,0))) AS -- !query SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) >= cast(cast(1 as string) as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -14919,7 +15007,7 @@ Project [(cast(1 as decimal(10,0)) >= cast(cast(1 as string) as decimal(10,0))) -- !query SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t -- !query analysis -Project [(cast(1 as decimal(20,0)) >= cast(cast(1 as string) as decimal(20,0))) AS (CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS STRING))#x] +Project [(cast(cast(1 as decimal(20,0)) as double) >= cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -16162,7 +16250,7 @@ Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as decimal(10,0)) as decima -- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(3,0)) = cast(cast(1 as string) as decimal(3,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING)))#x] +Project [NOT (cast(cast(1 as decimal(3,0)) as double) = cast(cast(1 as string) as double)) AS (NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -16173,7 +16261,7 @@ Project [NOT (cast(1 as decimal(3,0)) = cast(cast(1 as string) as decimal(3,0))) -- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(5,0)) = cast(cast(1 as string) as decimal(5,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING)))#x] +Project [NOT (cast(cast(1 as decimal(5,0)) as double) = cast(cast(1 as string) as double)) AS (NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -16184,7 +16272,7 @@ Project [NOT (cast(1 as decimal(5,0)) = cast(cast(1 as string) as decimal(5,0))) -- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as string) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING)))#x] +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(cast(1 as string) as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -16195,7 +16283,7 @@ Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as string) as decimal(10,0) -- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as string) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING)))#x] +Project [NOT (cast(cast(1 as decimal(20,0)) as double) = cast(cast(1 as string) as double)) AS (NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -16294,45 +16382,89 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(3,0)) = cast(cast(1 as boolean) as decimal(3,0))) AS (NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(5,0)) = cast(cast(1 as boolean) as decimal(5,0))) AS (NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(cast(1 as boolean) as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(20,0)) = cast(cast(1 as boolean) as decimal(20,0))) AS (NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN)))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) <> cast(1 as boolean)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out index 22b870bc0b420..f168b3221150e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/division.sql.out @@ -87,7 +87,7 @@ Project [(cast(cast(1 as tinyint) as decimal(3,0)) / cast(1 as decimal(10,0))) A -- !query SELECT cast(1 as tinyint) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) / cast(cast(cast(1 as string) as double) as double)) AS (CAST(1 AS TINYINT) / CAST(1 AS STRING))#x] +Project [(cast(cast(cast(1 as tinyint) as bigint) as double) / cast(cast(cast(1 as string) as bigint) as double)) AS (CAST(1 AS TINYINT) / CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -263,7 +263,7 @@ Project [(cast(cast(1 as smallint) as decimal(5,0)) / cast(1 as decimal(10,0))) -- !query SELECT cast(1 as smallint) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) / cast(cast(cast(1 as string) as double) as double)) AS (CAST(1 AS SMALLINT) / CAST(1 AS STRING))#x] +Project [(cast(cast(cast(1 as smallint) as bigint) as double) / cast(cast(cast(1 as string) as bigint) as double)) AS (CAST(1 AS SMALLINT) / CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -439,7 +439,7 @@ Project [(cast(cast(1 as int) as decimal(10,0)) / cast(1 as decimal(10,0))) AS ( -- !query SELECT cast(1 as int) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) / cast(cast(cast(1 as string) as double) as double)) AS (CAST(1 AS INT) / CAST(1 AS STRING))#x] +Project [(cast(cast(cast(1 as int) as bigint) as double) / cast(cast(cast(1 as string) as bigint) as double)) AS (CAST(1 AS INT) / CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -615,7 +615,7 @@ Project [(cast(cast(1 as bigint) as decimal(20,0)) / cast(1 as decimal(10,0))) A -- !query SELECT cast(1 as bigint) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as string) as double) as double)) AS (CAST(1 AS BIGINT) / CAST(1 AS STRING))#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as string) as bigint) as double)) AS (CAST(1 AS BIGINT) / CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -791,7 +791,7 @@ Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as decimal(10,0)) -- !query SELECT cast(1 as float) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as float) as double) / cast(cast(cast(1 as string) as double) as double)) AS (CAST(1 AS FLOAT) / CAST(1 AS STRING))#x] +Project [(cast(cast(1 as float) as double) / cast(cast(1 as string) as double)) AS (CAST(1 AS FLOAT) / CAST(1 AS STRING))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1242,7 +1242,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as string) / cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(cast(1 as string) as double) / cast(cast(1 as tinyint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS TINYINT))#x] +Project [(cast(cast(cast(1 as string) as bigint) as double) / cast(cast(cast(1 as tinyint) as bigint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1253,7 +1253,7 @@ Project [(cast(cast(1 as string) as double) / cast(cast(1 as tinyint) as double) -- !query SELECT cast(1 as string) / cast(1 as smallint) FROM t -- !query analysis -Project [(cast(cast(1 as string) as double) / cast(cast(1 as smallint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS SMALLINT))#x] +Project [(cast(cast(cast(1 as string) as bigint) as double) / cast(cast(cast(1 as smallint) as bigint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1264,7 +1264,7 @@ Project [(cast(cast(1 as string) as double) / cast(cast(1 as smallint) as double -- !query SELECT cast(1 as string) / cast(1 as int) FROM t -- !query analysis -Project [(cast(cast(1 as string) as double) / cast(cast(1 as int) as double)) AS (CAST(1 AS STRING) / CAST(1 AS INT))#x] +Project [(cast(cast(cast(1 as string) as bigint) as double) / cast(cast(cast(1 as int) as bigint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1275,7 +1275,7 @@ Project [(cast(cast(1 as string) as double) / cast(cast(1 as int) as double)) AS -- !query SELECT cast(1 as string) / cast(1 as bigint) FROM t -- !query analysis -Project [(cast(cast(1 as string) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS BIGINT))#x] +Project [(cast(cast(cast(1 as string) as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS STRING) / CAST(1 AS BIGINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1319,12 +1319,23 @@ Project [(cast(cast(1 as string) as double) / cast(cast(1 as decimal(10,0)) as d -- !query SELECT cast(1 as string) / cast(1 as string) FROM t -- !query analysis -Project [(cast(cast(1 as string) as double) / cast(cast(1 as string) as double)) AS (CAST(1 AS STRING) / CAST(1 AS STRING))#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as string) / cast(1 as string)" + } ] +} -- !query @@ -1332,11 +1343,11 @@ SELECT cast(1 as string) / cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -1354,11 +1365,11 @@ SELECT cast(1 as string) / cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -1376,11 +1387,11 @@ SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -1398,11 +1409,11 @@ SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -1574,11 +1585,11 @@ SELECT cast('1' as binary) / cast(1 as string) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -1838,11 +1849,11 @@ SELECT cast(1 as boolean) / cast(1 as string) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -2102,11 +2113,11 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -2366,11 +2377,11 @@ SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"" }, "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/elt.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/elt.sql.out index f4902012f0f96..f5b0740f2b462 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/elt.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/elt.sql.out @@ -11,7 +11,7 @@ FROM ( FROM range(10) ) -- !query analysis -Project [elt(2, col1#x, cast(col2#xL as string), col3#x, cast(col4#x as string), cast(col5#x as string), false) AS col#x] +Project [elt(2, col1#x, cast(col2#xL as string), col3#x, cast(col4#x as string), cast(col5#x as string), true) AS col#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [prefix_ AS col1#x, id#xL AS col2#xL, cast((id#xL + cast(1 as bigint)) as string) AS col3#x, encode(cast((id#xL + cast(2 as bigint)) as string), utf-8) AS col4#x, cast(id#xL as double) AS col5#x] +- Range (0, 10, step=1) @@ -28,7 +28,7 @@ FROM ( FROM range(10) ) -- !query analysis -Project [elt(3, col1#x, col2#x, cast(col3#x as string), cast(col4#x as string), false) AS col#x] +Project [elt(3, col1#x, col2#x, cast(col3#x as string), cast(col4#x as string), true) AS col#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [cast(id#xL as string) AS col1#x, cast((id#xL + cast(1 as bigint)) as string) AS col2#x, encode(cast((id#xL + cast(2 as bigint)) as string), utf-8) AS col3#x, encode(cast((id#xL + cast(3 as bigint)) as string), utf-8) AS col4#x] +- Range (0, 10, step=1) @@ -49,7 +49,7 @@ FROM ( FROM range(10) ) -- !query analysis -Project [elt(1, cast(col1#x as string), cast(col2#x as string), false) AS col#x] +Project [elt(1, cast(col1#x as string), cast(col2#x as string), true) AS col#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [encode(cast(id#xL as string), utf-8) AS col1#x, encode(cast((id#xL + cast(1 as bigint)) as string), utf-8) AS col2#x] +- Range (0, 10, step=1) @@ -70,7 +70,7 @@ FROM ( FROM range(10) ) -- !query analysis -Project [elt(2, col1#x, col2#x, false) AS col#x] +Project [elt(2, col1#x, col2#x, true) AS col#x] +- SubqueryAlias __auto_generated_subquery_name +- Project [encode(cast(id#xL as string), utf-8) AS col1#x, encode(cast((id#xL + cast(1 as bigint)) as string), utf-8) AS col2#x] +- Range (0, 10, step=1) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out index b1d07bd7be902..791b75a869718 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/ifCoercion.sql.out @@ -54,7 +54,7 @@ Project [if (true) cast(cast(1 as tinyint) as bigint) else cast(2 as bigint) AS -- !query SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as tinyint) as float) else cast(2 as float) AS (IF(true, CAST(1 AS TINYINT), CAST(2 AS FLOAT)))#x] +Project [if (true) cast(cast(1 as tinyint) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS TINYINT), CAST(2 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -87,7 +87,7 @@ Project [if (true) cast(cast(1 as tinyint) as decimal(10,0)) else cast(2 as deci -- !query SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as tinyint) as string) else cast(2 as string) AS (IF(true, CAST(1 AS TINYINT), CAST(2 AS STRING)))#x] +Project [if (true) cast(cast(1 as tinyint) as bigint) else cast(cast(2 as string) as bigint) AS (IF(true, CAST(1 AS TINYINT), CAST(2 AS STRING)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -230,7 +230,7 @@ Project [if (true) cast(cast(1 as smallint) as bigint) else cast(2 as bigint) AS -- !query SELECT IF(true, cast(1 as smallint), cast(2 as float)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as smallint) as float) else cast(2 as float) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 AS FLOAT)))#x] +Project [if (true) cast(cast(1 as smallint) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -263,7 +263,7 @@ Project [if (true) cast(cast(1 as smallint) as decimal(10,0)) else cast(2 as dec -- !query SELECT IF(true, cast(1 as smallint), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as smallint) as string) else cast(2 as string) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 AS STRING)))#x] +Project [if (true) cast(cast(1 as smallint) as bigint) else cast(cast(2 as string) as bigint) AS (IF(true, CAST(1 AS SMALLINT), CAST(2 AS STRING)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -406,7 +406,7 @@ Project [if (true) cast(cast(1 as int) as bigint) else cast(2 as bigint) AS (IF( -- !query SELECT IF(true, cast(1 as int), cast(2 as float)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as int) as float) else cast(2 as float) AS (IF(true, CAST(1 AS INT), CAST(2 AS FLOAT)))#x] +Project [if (true) cast(cast(1 as int) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS INT), CAST(2 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -439,7 +439,7 @@ Project [if (true) cast(cast(1 as int) as decimal(10,0)) else cast(2 as decimal( -- !query SELECT IF(true, cast(1 as int), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as int) as string) else cast(2 as string) AS (IF(true, CAST(1 AS INT), CAST(2 AS STRING)))#x] +Project [if (true) cast(cast(1 as int) as bigint) else cast(cast(2 as string) as bigint) AS (IF(true, CAST(1 AS INT), CAST(2 AS STRING)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -582,7 +582,7 @@ Project [if (true) cast(1 as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 -- !query SELECT IF(true, cast(1 as bigint), cast(2 as float)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as bigint) as float) else cast(2 as float) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS FLOAT)))#x] +Project [if (true) cast(cast(1 as bigint) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -615,7 +615,7 @@ Project [if (true) cast(cast(1 as bigint) as decimal(20,0)) else cast(cast(2 as -- !query SELECT IF(true, cast(1 as bigint), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as bigint) as string) else cast(2 as string) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS STRING)))#x] +Project [if (true) cast(1 as bigint) else cast(cast(2 as string) as bigint) AS (IF(true, CAST(1 AS BIGINT), CAST(2 AS STRING)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -714,7 +714,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as float), cast(2 as tinyint)) FROM t -- !query analysis -Project [if (true) cast(1 as float) else cast(cast(2 as tinyint) as float) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT)))#x] +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as tinyint) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -725,7 +725,7 @@ Project [if (true) cast(1 as float) else cast(cast(2 as tinyint) as float) AS (I -- !query SELECT IF(true, cast(1 as float), cast(2 as smallint)) FROM t -- !query analysis -Project [if (true) cast(1 as float) else cast(cast(2 as smallint) as float) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT)))#x] +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as smallint) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -736,7 +736,7 @@ Project [if (true) cast(1 as float) else cast(cast(2 as smallint) as float) AS ( -- !query SELECT IF(true, cast(1 as float), cast(2 as int)) FROM t -- !query analysis -Project [if (true) cast(1 as float) else cast(cast(2 as int) as float) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS INT)))#x] +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as int) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -747,7 +747,7 @@ Project [if (true) cast(1 as float) else cast(cast(2 as int) as float) AS (IF(tr -- !query SELECT IF(true, cast(1 as float), cast(2 as bigint)) FROM t -- !query analysis -Project [if (true) cast(1 as float) else cast(cast(2 as bigint) as float) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS BIGINT)))#x] +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as bigint) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS BIGINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -791,7 +791,7 @@ Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as decimal( -- !query SELECT IF(true, cast(1 as float), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as float) as string) else cast(2 as string) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS STRING)))#x] +Project [if (true) cast(cast(1 as float) as double) else cast(cast(2 as string) as double) AS (IF(true, CAST(1 AS FLOAT), CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -967,7 +967,7 @@ Project [if (true) cast(1 as double) else cast(cast(2 as decimal(10,0)) as doubl -- !query SELECT IF(true, cast(1 as double), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as double) as string) else cast(2 as string) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 AS STRING)))#x] +Project [if (true) cast(1 as double) else cast(cast(2 as string) as double) AS (IF(true, CAST(1 AS DOUBLE), CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1143,7 +1143,7 @@ Project [if (true) cast(1 as decimal(10,0)) else cast(2 as decimal(10,0)) AS (IF -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(1 as decimal(10,0)) as string) else cast(2 as string) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS STRING)))#x] +Project [if (true) cast(cast(1 as decimal(10,0)) as double) else cast(cast(2 as string) as double) AS (IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1242,7 +1242,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as string), cast(2 as tinyint)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as tinyint) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT)))#x] +Project [if (true) cast(cast(1 as string) as bigint) else cast(cast(2 as tinyint) as bigint) AS (IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1253,7 +1253,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as tinyint) as string) AS -- !query SELECT IF(true, cast(1 as string), cast(2 as smallint)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as smallint) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT)))#x] +Project [if (true) cast(cast(1 as string) as bigint) else cast(cast(2 as smallint) as bigint) AS (IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1264,7 +1264,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as smallint) as string) AS -- !query SELECT IF(true, cast(1 as string), cast(2 as int)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as int) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS INT)))#x] +Project [if (true) cast(cast(1 as string) as bigint) else cast(cast(2 as int) as bigint) AS (IF(true, CAST(1 AS STRING), CAST(2 AS INT)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1275,7 +1275,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as int) as string) AS (IF( -- !query SELECT IF(true, cast(1 as string), cast(2 as bigint)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as bigint) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT)))#x] +Project [if (true) cast(cast(1 as string) as bigint) else cast(2 as bigint) AS (IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT)))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1286,7 +1286,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as bigint) as string) AS ( -- !query SELECT IF(true, cast(1 as string), cast(2 as float)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as float) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS FLOAT)))#x] +Project [if (true) cast(cast(1 as string) as double) else cast(cast(2 as float) as double) AS (IF(true, CAST(1 AS STRING), CAST(2 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1297,7 +1297,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as float) as string) AS (I -- !query SELECT IF(true, cast(1 as string), cast(2 as double)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as double) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS DOUBLE)))#x] +Project [if (true) cast(cast(1 as string) as double) else cast(2 as double) AS (IF(true, CAST(1 AS STRING), CAST(2 AS DOUBLE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1308,7 +1308,7 @@ Project [if (true) cast(1 as string) else cast(cast(2 as double) as string) AS ( -- !query SELECT IF(true, cast(1 as string), cast(2 as decimal(10, 0))) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2 as decimal(10,0)) as string) AS (IF(true, CAST(1 AS STRING), CAST(2 AS DECIMAL(10,0))))#x] +Project [if (true) cast(cast(1 as string) as double) else cast(cast(2 as decimal(10,0)) as double) AS (IF(true, CAST(1 AS STRING), CAST(2 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1330,51 +1330,29 @@ Project [if (true) cast(1 as string) else cast(2 as string) AS (IF(true, CAST(1 -- !query SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BINARY)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "IF(true, cast(1 as string), cast('2' as binary))" - } ] -} +Project [if (true) cast(cast(1 as string) as binary) else cast(2 as binary) AS (IF(true, CAST(1 AS STRING), CAST(2 AS BINARY)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "IF(true, cast(1 as string), cast(2 as boolean))" - } ] -} +Project [if (true) cast(cast(1 as string) as boolean) else cast(2 as boolean) AS (IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2017-12-11 09:30:00.0 as timestamp) as string) AS (IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +Project [if (true) cast(cast(1 as string) as timestamp) else cast(2017-12-11 09:30:00.0 as timestamp) AS (IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1385,7 +1363,7 @@ Project [if (true) cast(1 as string) else cast(cast(2017-12-11 09:30:00.0 as tim -- !query SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -Project [if (true) cast(1 as string) else cast(cast(2017-12-11 09:30:00 as date) as string) AS (IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE)))#x] +Project [if (true) cast(cast(1 as string) as date) else cast(2017-12-11 09:30:00 as date) AS (IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1550,23 +1528,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "IF(true, cast('1' as binary), cast(2 as string))" - } ] -} +Project [if (true) cast(1 as binary) else cast(cast(2 as string) as binary) AS (IF(true, CAST(1 AS BINARY), CAST(2 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1803,23 +1770,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "IF(true, cast(1 as boolean), cast(2 as string))" - } ] -} +Project [if (true) cast(1 as boolean) else cast(cast(2 as string) as boolean) AS (IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2056,7 +2012,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) else cast(2 as string) AS (IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS STRING)))#x] +Project [if (true) cast(2017-12-12 09:30:00.0 as timestamp) else cast(cast(2 as string) as timestamp) AS (IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2287,7 +2243,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as string)) FROM t -- !query analysis -Project [if (true) cast(cast(2017-12-12 09:30:00 as date) as string) else cast(2 as string) AS (IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS STRING)))#x] +Project [if (true) cast(2017-12-12 09:30:00 as date) else cast(cast(2 as string) as date) AS (IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/implicitTypeCasts.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/implicitTypeCasts.sql.out index 43aaea63fd045..977b1e1459c3e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/implicitTypeCasts.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/implicitTypeCasts.sql.out @@ -10,7 +10,7 @@ CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, UNSUPPORTED, true -- !query SELECT 1 + '2' FROM t -- !query analysis -Project [(cast(1 as double) + cast(2 as double)) AS (1 + 2)#x] +Project [(cast(1 as bigint) + cast(2 as bigint)) AS (1 + 2)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -21,7 +21,7 @@ Project [(cast(1 as double) + cast(2 as double)) AS (1 + 2)#x] -- !query SELECT 1 - '2' FROM t -- !query analysis -Project [(cast(1 as double) - cast(2 as double)) AS (1 - 2)#x] +Project [(cast(1 as bigint) - cast(2 as bigint)) AS (1 - 2)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -32,7 +32,7 @@ Project [(cast(1 as double) - cast(2 as double)) AS (1 - 2)#x] -- !query SELECT 1 * '2' FROM t -- !query analysis -Project [(cast(1 as double) * cast(2 as double)) AS (1 * 2)#x] +Project [(cast(1 as bigint) * cast(2 as bigint)) AS (1 * 2)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -43,7 +43,7 @@ Project [(cast(1 as double) * cast(2 as double)) AS (1 * 2)#x] -- !query SELECT 4 / '2' FROM t -- !query analysis -Project [(cast(4 as double) / cast(cast(2 as double) as double)) AS (4 / 2)#x] +Project [(cast(cast(4 as bigint) as double) / cast(cast(2 as bigint) as double)) AS (4 / 2)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out index 0db96719a3fb0..71bc2fef3ab8e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/inConversion.sql.out @@ -54,7 +54,7 @@ Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as bigint) as bigint -- !query SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as tinyint) as float) IN (cast(cast(1 as float) as float)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as tinyint) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -87,7 +87,7 @@ Project [cast(cast(1 as tinyint) as decimal(10,0)) IN (cast(cast(1 as decimal(10 -- !query SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as tinyint) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as string) as bigint)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -230,7 +230,7 @@ Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as bigint) as bigin -- !query SELECT cast(1 as smallint) in (cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as smallint) as float) IN (cast(cast(1 as float) as float)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as smallint) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -263,7 +263,7 @@ Project [cast(cast(1 as smallint) as decimal(10,0)) IN (cast(cast(1 as decimal(1 -- !query SELECT cast(1 as smallint) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as smallint) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as string) as bigint)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -406,7 +406,7 @@ Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as bigint) as bigint)) A -- !query SELECT cast(1 as int) in (cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as int) as float) IN (cast(cast(1 as float) as float)) AS (CAST(1 AS INT) IN (CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as int) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS INT) IN (CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -439,7 +439,7 @@ Project [cast(cast(1 as int) as decimal(10,0)) IN (cast(cast(1 as decimal(10,0)) -- !query SELECT cast(1 as int) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as int) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS INT) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as string) as bigint)) AS (CAST(1 AS INT) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -582,7 +582,7 @@ Project [cast(1 as bigint) IN (cast(1 as bigint)) AS (CAST(1 AS BIGINT) IN (CAST -- !query SELECT cast(1 as bigint) in (cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as bigint) as float) IN (cast(cast(1 as float) as float)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as bigint) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -615,7 +615,7 @@ Project [cast(cast(1 as bigint) as decimal(20,0)) IN (cast(cast(1 as decimal(10, -- !query SELECT cast(1 as bigint) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as bigint) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as string) as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -714,7 +714,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as float) in (cast(1 as tinyint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as tinyint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS TINYINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as tinyint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -725,7 +725,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as tinyint) as float)) -- !query SELECT cast(1 as float) in (cast(1 as smallint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as smallint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS SMALLINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as smallint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -736,7 +736,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as smallint) as float)) -- !query SELECT cast(1 as float) in (cast(1 as int)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as int) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS INT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as int) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -747,7 +747,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as int) as float)) AS ( -- !query SELECT cast(1 as float) in (cast(1 as bigint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as bigint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS BIGINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as bigint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS BIGINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -791,7 +791,7 @@ Project [cast(cast(1 as float) as double) IN (cast(cast(1 as decimal(10,0)) as d -- !query SELECT cast(1 as float) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as float) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as string) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -967,7 +967,7 @@ Project [cast(cast(1 as double) as double) IN (cast(cast(1 as decimal(10,0)) as -- !query SELECT cast(1 as double) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as double) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as string) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1143,7 +1143,7 @@ Project [cast(1 as decimal(10,0)) IN (cast(1 as decimal(10,0))) AS (CAST(1 AS DE -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as decimal(10,0)) as string) IN (cast(cast(1 as string) as string)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS STRING)))#x] +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1242,7 +1242,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as string) in (cast(1 as tinyint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as tinyint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS TINYINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as tinyint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1253,7 +1253,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as tinyint) as string -- !query SELECT cast(1 as string) in (cast(1 as smallint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as smallint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS SMALLINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as smallint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1264,7 +1264,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as smallint) as strin -- !query SELECT cast(1 as string) in (cast(1 as int)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as int) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS INT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as int) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1275,7 +1275,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as int) as string)) A -- !query SELECT cast(1 as string) in (cast(1 as bigint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as bigint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS BIGINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS BIGINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1286,7 +1286,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as bigint) as string) -- !query SELECT cast(1 as string) in (cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as float) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as float) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1297,7 +1297,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as float) as string)) -- !query SELECT cast(1 as string) in (cast(1 as double)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as double) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS DOUBLE)))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as double) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS DOUBLE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1308,7 +1308,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as double) as string) -- !query SELECT cast(1 as string) in (cast(1 as decimal(10, 0))) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as decimal(10,0)) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS DECIMAL(10,0))))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1330,51 +1330,29 @@ Project [cast(1 as string) IN (cast(1 as string)) AS (CAST(1 AS STRING) IN (CAST -- !query SELECT cast(1 as string) in (cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BINARY)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 49, - "fragment" : "in (cast('1' as binary))" - } ] -} +Project [cast(cast(1 as string) as binary) IN (cast(cast(1 as binary) as binary)) AS (CAST(1 AS STRING) IN (CAST(1 AS BINARY)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as string) in (cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 48, - "fragment" : "in (cast(1 as boolean))" - } ] -} +Project [cast(cast(1 as string) as boolean) IN (cast(cast(1 as boolean) as boolean)) AS (CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(2017-12-11 09:30:00.0 as timestamp) as string)) AS (CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +Project [cast(cast(1 as string) as timestamp) IN (cast(cast(2017-12-11 09:30:00.0 as timestamp) as timestamp)) AS (CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1385,7 +1363,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(2017-12-11 09:30:00.0 a -- !query SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(2017-12-11 09:30:00 as date) as string)) AS (CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00 AS DATE)))#x] +Project [cast(cast(1 as string) as date) IN (cast(cast(2017-12-11 09:30:00 as date) as date)) AS (CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00 AS DATE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1550,23 +1528,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 49, - "fragment" : "in (cast(1 as string))" - } ] -} +Project [cast(cast(1 as binary) as binary) IN (cast(cast(1 as string) as binary)) AS (CAST(1 AS BINARY) IN (CAST(1 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1803,23 +1770,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT true in (cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(true IN (CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 34, - "fragment" : "in (cast(1 as string))" - } ] -} +Project [cast(true as boolean) IN (cast(cast(1 as string) as boolean)) AS (true IN (CAST(1 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2056,7 +2012,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as string)) FROM t -- !query analysis -Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) IN (cast(cast(2 as string) as string)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS STRING)))#x] +Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp) IN (cast(cast(2 as string) as timestamp)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2287,7 +2243,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as string)) FROM t -- !query analysis -Project [cast(cast(2017-12-12 09:30:00 as date) as string) IN (cast(cast(2 as string) as string)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS STRING)))#x] +Project [cast(cast(2017-12-12 09:30:00 as date) as date) IN (cast(cast(2 as string) as date)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2408,7 +2364,7 @@ Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as tinyint) as bigin -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as tinyint) as float) IN (cast(cast(1 as tinyint) as float),cast(cast(1 as float) as float)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as tinyint) as double) IN (cast(cast(1 as tinyint) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2441,7 +2397,7 @@ Project [cast(cast(1 as tinyint) as decimal(10,0)) IN (cast(cast(1 as tinyint) a -- !query SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as tinyint) as string) IN (cast(cast(1 as tinyint) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as tinyint) as bigint) IN (cast(cast(1 as tinyint) as bigint),cast(cast(1 as string) as bigint)) AS (CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2584,7 +2540,7 @@ Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as smallint) as big -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as smallint) as float) IN (cast(cast(1 as smallint) as float),cast(cast(1 as float) as float)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as smallint) as double) IN (cast(cast(1 as smallint) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2617,7 +2573,7 @@ Project [cast(cast(1 as smallint) as decimal(10,0)) IN (cast(cast(1 as smallint) -- !query SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as smallint) as string) IN (cast(cast(1 as smallint) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as smallint) as bigint) IN (cast(cast(1 as smallint) as bigint),cast(cast(1 as string) as bigint)) AS (CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2760,7 +2716,7 @@ Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as int) as bigint),cast( -- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as int) as float) IN (cast(cast(1 as int) as float),cast(cast(1 as float) as float)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as int) as double) IN (cast(cast(1 as int) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2793,7 +2749,7 @@ Project [cast(cast(1 as int) as decimal(10,0)) IN (cast(cast(1 as int) as decima -- !query SELECT cast(1 as int) in (cast(1 as int), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as int) as string) IN (cast(cast(1 as int) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as int) as bigint) IN (cast(cast(1 as int) as bigint),cast(cast(1 as string) as bigint)) AS (CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2936,7 +2892,7 @@ Project [cast(1 as bigint) IN (cast(1 as bigint),cast(1 as bigint)) AS (CAST(1 A -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as bigint) as float) IN (cast(cast(1 as bigint) as float),cast(cast(1 as float) as float)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as bigint) as double) IN (cast(cast(1 as bigint) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2969,7 +2925,7 @@ Project [cast(cast(1 as bigint) as decimal(20,0)) IN (cast(cast(1 as bigint) as -- !query SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as bigint) as string) IN (cast(cast(1 as bigint) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as bigint) as bigint) IN (cast(cast(1 as bigint) as bigint),cast(cast(1 as string) as bigint)) AS (CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3068,7 +3024,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as tinyint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cast(cast(1 as tinyint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS TINYINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as tinyint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3079,7 +3035,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cas -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as smallint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cast(cast(1 as smallint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS SMALLINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as smallint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3090,7 +3046,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cas -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as int)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cast(cast(1 as int) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS INT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as int) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3101,7 +3057,7 @@ Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cas -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as bigint)) FROM t -- !query analysis -Project [cast(cast(1 as float) as float) IN (cast(cast(1 as float) as float),cast(cast(1 as bigint) as float)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BIGINT)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as bigint) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BIGINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3145,7 +3101,7 @@ Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),c -- !query SELECT cast(1 as float) in (cast(1 as float), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as float) as string) IN (cast(cast(1 as float) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as float) as double) IN (cast(cast(1 as float) as double),cast(cast(1 as string) as double)) AS (CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3321,7 +3277,7 @@ Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double) -- !query SELECT cast(1 as double) in (cast(1 as double), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as double) as string) IN (cast(cast(1 as double) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as double) as double) IN (cast(cast(1 as double) as double),cast(cast(1 as string) as double)) AS (CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3497,7 +3453,7 @@ Project [cast(1 as decimal(10,0)) IN (cast(1 as decimal(10,0)),cast(1 as decimal -- !query SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(1 as decimal(10,0)) as string) IN (cast(cast(1 as decimal(10,0)) as string),cast(cast(1 as string) as string)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS STRING)))#x] +Project [cast(cast(1 as decimal(10,0)) as double) IN (cast(cast(1 as decimal(10,0)) as double),cast(cast(1 as string) as double)) AS (CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3596,7 +3552,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as tinyint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as tinyint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS TINYINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as string) as bigint),cast(cast(1 as tinyint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3607,7 +3563,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as smallint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as smallint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS SMALLINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as string) as bigint),cast(cast(1 as smallint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3618,7 +3574,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as int)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as int) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS INT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as string) as bigint),cast(cast(1 as int) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3629,7 +3585,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as bigint)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as bigint) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BIGINT)))#x] +Project [cast(cast(1 as string) as bigint) IN (cast(cast(1 as string) as bigint),cast(cast(1 as bigint) as bigint)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BIGINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3640,7 +3596,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as float)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as float) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS FLOAT)))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as string) as double),cast(cast(1 as float) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3651,7 +3607,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as double)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as double) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DOUBLE)))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as string) as double),cast(cast(1 as double) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DOUBLE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3662,7 +3618,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as decimal(10, 0))) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(1 as decimal(10,0)) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DECIMAL(10,0))))#x] +Project [cast(cast(1 as string) as double) IN (cast(cast(1 as string) as double),cast(cast(1 as decimal(10,0)) as double)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3684,51 +3640,29 @@ Project [cast(1 as string) IN (cast(1 as string),cast(1 as string)) AS (CAST(1 A -- !query SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"STRING\", \"BINARY\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 68, - "fragment" : "in (cast(1 as string), cast('1' as binary))" - } ] -} +Project [cast(cast(1 as string) as binary) IN (cast(cast(1 as string) as binary),cast(cast(1 as binary) as binary)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"STRING\", \"STRING\", \"BOOLEAN\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 67, - "fragment" : "in (cast(1 as string), cast(1 as boolean))" - } ] -} +Project [cast(cast(1 as string) as boolean) IN (cast(cast(1 as string) as boolean),cast(cast(1 as boolean) as boolean)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(2017-12-11 09:30:00.0 as timestamp) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +Project [cast(cast(1 as string) as timestamp) IN (cast(cast(1 as string) as timestamp),cast(cast(2017-12-11 09:30:00.0 as timestamp) as timestamp)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3739,7 +3673,7 @@ Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string) -- !query SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis -Project [cast(cast(1 as string) as string) IN (cast(cast(1 as string) as string),cast(cast(2017-12-11 09:30:00 as date) as string)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE)))#x] +Project [cast(cast(1 as string) as date) IN (cast(cast(1 as string) as date),cast(cast(2017-12-11 09:30:00 as date) as date)) AS (CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3904,23 +3838,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"BINARY\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 70, - "fragment" : "in (cast('1' as binary), cast(1 as string))" - } ] -} +Project [cast(cast(1 as binary) as binary) IN (cast(cast(1 as binary) as binary),cast(cast(1 as string) as binary)) AS (CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -4157,23 +4080,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 29, - "stopIndex" : 72, - "fragment" : "in (cast('1' as boolean), cast(1 as string))" - } ] -} +Project [cast(cast(1 as boolean) as boolean) IN (cast(cast(1 as boolean) as boolean),cast(cast(1 as string) as boolean)) AS (CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING)))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -4410,7 +4322,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as string) IN (cast(cast(2017-12-12 09:30:00.0 as timestamp) as string),cast(cast(1 as string) as string)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS STRING)))#x] +Project [cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp) IN (cast(cast(2017-12-12 09:30:00.0 as timestamp) as timestamp),cast(cast(1 as string) as timestamp)) AS (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -4641,7 +4553,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as string)) FROM t -- !query analysis -Project [cast(cast(2017-12-12 09:30:00 as date) as string) IN (cast(cast(2017-12-12 09:30:00 as date) as string),cast(cast(1 as string) as string)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS STRING)))#x] +Project [cast(cast(2017-12-12 09:30:00 as date) as date) IN (cast(cast(2017-12-12 09:30:00 as date) as date),cast(cast(1 as string) as date)) AS (CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS STRING)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out index 0f72b0cf8a0e3..13cca708b8cbc 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapZipWith.sql.out @@ -193,52 +193,96 @@ Project [map_zip_with(cast(decimal_map2#x as map), double SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -Project [map_zip_with(string_map1#x, cast(int_map#x as map), lambdafunction(struct(k, lambda k#x, v1, lambda v1#x, v2, lambda v2#x), lambda k#x, lambda v1#x, lambda v2#x, false)) AS m#x] -+- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x]) - +- Project [cast(boolean_map#x as map) AS boolean_map#x, cast(tinyint_map#x as map) AS tinyint_map#x, cast(smallint_map#x as map) AS smallint_map#x, cast(int_map#x as map) AS int_map#x, cast(bigint_map#x as map) AS bigint_map#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map#x as map) AS double_map#x, cast(float_map#x as map) AS float_map#x, cast(date_map#x as map) AS date_map#x, cast(timestamp_map#x as map) AS timestamp_map#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(string_map3#x as map) AS string_map3#x, cast(string_map4#x as map) AS string_map4#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, cast(struct_map1#x as map,struct>) AS struct_map1#x, cast(struct_map2#x as map,struct>) AS struct_map2#x] - +- Project [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] - +- SubqueryAlias various_maps - +- LocalRelation [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_zip_with(string_map1, int_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -Project [map_zip_with(string_map2#x, cast(date_map#x as map), lambdafunction(struct(k, lambda k#x, v1, lambda v1#x, v2, lambda v2#x), lambda k#x, lambda v1#x, lambda v2#x, false)) AS m#x] -+- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x]) - +- Project [cast(boolean_map#x as map) AS boolean_map#x, cast(tinyint_map#x as map) AS tinyint_map#x, cast(smallint_map#x as map) AS smallint_map#x, cast(int_map#x as map) AS int_map#x, cast(bigint_map#x as map) AS bigint_map#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map#x as map) AS double_map#x, cast(float_map#x as map) AS float_map#x, cast(date_map#x as map) AS date_map#x, cast(timestamp_map#x as map) AS timestamp_map#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(string_map3#x as map) AS string_map3#x, cast(string_map4#x as map) AS string_map4#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, cast(struct_map1#x as map,struct>) AS struct_map1#x, cast(struct_map2#x as map,struct>) AS struct_map2#x] - +- Project [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] - +- SubqueryAlias various_maps - +- LocalRelation [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"DATE\"", + "sqlExpr" : "\"map_zip_with(string_map2, date_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -Project [map_zip_with(cast(timestamp_map#x as map), string_map3#x, lambdafunction(struct(k, lambda k#x, v1, lambda v1#x, v2, lambda v2#x), lambda k#x, lambda v1#x, lambda v2#x, false)) AS m#x] -+- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x]) - +- Project [cast(boolean_map#x as map) AS boolean_map#x, cast(tinyint_map#x as map) AS tinyint_map#x, cast(smallint_map#x as map) AS smallint_map#x, cast(int_map#x as map) AS int_map#x, cast(bigint_map#x as map) AS bigint_map#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map#x as map) AS double_map#x, cast(float_map#x as map) AS float_map#x, cast(date_map#x as map) AS date_map#x, cast(timestamp_map#x as map) AS timestamp_map#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(string_map3#x as map) AS string_map3#x, cast(string_map4#x as map) AS string_map4#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, cast(struct_map1#x as map,struct>) AS struct_map1#x, cast(struct_map2#x as map,struct>) AS struct_map2#x] - +- Project [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] - +- SubqueryAlias various_maps - +- LocalRelation [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"TIMESTAMP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(timestamp_map, string_map3, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query analysis -Project [map_zip_with(cast(decimal_map1#x as map), string_map4#x, lambdafunction(struct(k, lambda k#x, v1, lambda v1#x, v2, lambda v2#x), lambda k#x, lambda v1#x, lambda v2#x, false)) AS m#x] -+- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x]) - +- Project [cast(boolean_map#x as map) AS boolean_map#x, cast(tinyint_map#x as map) AS tinyint_map#x, cast(smallint_map#x as map) AS smallint_map#x, cast(int_map#x as map) AS int_map#x, cast(bigint_map#x as map) AS bigint_map#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map#x as map) AS double_map#x, cast(float_map#x as map) AS float_map#x, cast(date_map#x as map) AS date_map#x, cast(timestamp_map#x as map) AS timestamp_map#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(string_map3#x as map) AS string_map3#x, cast(string_map4#x as map) AS string_map4#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, cast(struct_map1#x as map,struct>) AS struct_map1#x, cast(struct_map2#x as map,struct>) AS struct_map2#x] - +- Project [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] - +- SubqueryAlias various_maps - +- LocalRelation [boolean_map#x, tinyint_map#x, smallint_map#x, int_map#x, bigint_map#x, decimal_map1#x, decimal_map2#x, double_map#x, float_map#x, date_map#x, timestamp_map#x, string_map1#x, string_map2#x, string_map3#x, string_map4#x, array_map1#x, array_map2#x, struct_map1#x, struct_map2#x] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"DECIMAL(36,0)\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(decimal_map1, string_map4, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out index dd3e56fe9322d..b1a3dc46dabac 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/mapconcat.sql.out @@ -117,13 +117,23 @@ SELECT map_concat(int_string_map1, tinyint_map2) istt_map FROM various_maps -- !query analysis -Project [map_concat(cast(tinyint_map1#x as map), smallint_map2#x) AS ts_map#x, map_concat(cast(smallint_map1#x as map), int_map2#x) AS si_map#x, map_concat(cast(int_map1#x as map), bigint_map2#x) AS ib_map#x, map_concat(cast(bigint_map1#x as map), cast(decimal_map2#x as map)) AS bd_map#x, map_concat(cast(decimal_map1#x as map), cast(float_map2#x as map)) AS df_map#x, map_concat(string_map1#x, cast(date_map2#x as map)) AS std_map#x, map_concat(cast(timestamp_map1#x as map), string_map2#x) AS tst_map#x, map_concat(string_map1#x, cast(int_map2#x as map)) AS sti_map#x, map_concat(int_string_map1#x, cast(tinyint_map2#x as map)) AS istt_map#x] -+- SubqueryAlias various_maps - +- View (`various_maps`, [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields]) - +- Project [cast(boolean_map1#x as map) AS boolean_map1#x, cast(boolean_map2#x as map) AS boolean_map2#x, cast(tinyint_map1#x as map) AS tinyint_map1#x, cast(tinyint_map2#x as map) AS tinyint_map2#x, cast(smallint_map1#x as map) AS smallint_map1#x, cast(smallint_map2#x as map) AS smallint_map2#x, cast(int_map1#x as map) AS int_map1#x, cast(int_map2#x as map) AS int_map2#x, cast(bigint_map1#x as map) AS bigint_map1#x, cast(bigint_map2#x as map) AS bigint_map2#x, cast(decimal_map1#x as map) AS decimal_map1#x, cast(decimal_map2#x as map) AS decimal_map2#x, cast(double_map1#x as map) AS double_map1#x, cast(double_map2#x as map) AS double_map2#x, cast(float_map1#x as map) AS float_map1#x, cast(float_map2#x as map) AS float_map2#x, cast(date_map1#x as map) AS date_map1#x, cast(date_map2#x as map) AS date_map2#x, cast(timestamp_map1#x as map) AS timestamp_map1#x, cast(timestamp_map2#x as map) AS timestamp_map2#x, cast(string_map1#x as map) AS string_map1#x, cast(string_map2#x as map) AS string_map2#x, cast(array_map1#x as map,array>) AS array_map1#x, cast(array_map2#x as map,array>) AS array_map2#x, ... 6 more fields] - +- Project [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] - +- SubqueryAlias various_maps - +- LocalRelation [boolean_map1#x, boolean_map2#x, tinyint_map1#x, tinyint_map2#x, smallint_map1#x, smallint_map2#x, int_map1#x, int_map2#x, bigint_map1#x, bigint_map2#x, decimal_map1#x, decimal_map2#x, double_map1#x, double_map2#x, float_map1#x, float_map2#x, date_map1#x, date_map2#x, timestamp_map1#x, timestamp_map2#x, string_map1#x, string_map2#x, array_map1#x, array_map2#x, ... 6 more fields] +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(string_map1, date_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 257, + "stopIndex" : 290, + "fragment" : "map_concat(string_map1, date_map2)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out index ccd34cfaeb67f..a3420c3cb0635 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/promoteStrings.sql.out @@ -10,7 +10,7 @@ CreateViewCommand `t`, SELECT 1, false, false, LocalTempView, UNSUPPORTED, true -- !query SELECT '1' + cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as double) + cast(cast(1 as tinyint) as double)) AS (1 + CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) + cast(cast(1 as tinyint) as bigint)) AS (1 + CAST(1 AS TINYINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -21,7 +21,7 @@ Project [(cast(1 as double) + cast(cast(1 as tinyint) as double)) AS (1 + CAST(1 -- !query SELECT '1' + cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as double) + cast(cast(1 as smallint) as double)) AS (1 + CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) + cast(cast(1 as smallint) as bigint)) AS (1 + CAST(1 AS SMALLINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -32,7 +32,7 @@ Project [(cast(1 as double) + cast(cast(1 as smallint) as double)) AS (1 + CAST( -- !query SELECT '1' + cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as double) + cast(cast(1 as int) as double)) AS (1 + CAST(1 AS INT))#x] +Project [(cast(1 as bigint) + cast(cast(1 as int) as bigint)) AS (1 + CAST(1 AS INT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -43,7 +43,7 @@ Project [(cast(1 as double) + cast(cast(1 as int) as double)) AS (1 + CAST(1 AS -- !query SELECT '1' + cast(1 as bigint) FROM t -- !query analysis -Project [(cast(1 as double) + cast(cast(1 as bigint) as double)) AS (1 + CAST(1 AS BIGINT))#x] +Project [(cast(1 as bigint) + cast(1 as bigint)) AS (1 + CAST(1 AS BIGINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -87,12 +87,23 @@ Project [(cast(1 as double) + cast(cast(1 as decimal(10,0)) as double)) AS (1 + -- !query SELECT '1' + '1' FROM t -- !query analysis -Project [(cast(1 as double) + cast(1 as double)) AS (1 + 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' + '1'" + } ] +} -- !query @@ -100,11 +111,11 @@ SELECT '1' + cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -122,11 +133,11 @@ SELECT '1' + cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -144,11 +155,11 @@ SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -169,11 +180,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"CAST(2017-12-11 09:30:00 AS DATE)\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + "sqlExpr" : "\"date_add(1, CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", @@ -188,7 +199,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' - cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as double) - cast(cast(1 as tinyint) as double)) AS (1 - CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) - cast(cast(1 as tinyint) as bigint)) AS (1 - CAST(1 AS TINYINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -199,7 +210,7 @@ Project [(cast(1 as double) - cast(cast(1 as tinyint) as double)) AS (1 - CAST(1 -- !query SELECT '1' - cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as double) - cast(cast(1 as smallint) as double)) AS (1 - CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) - cast(cast(1 as smallint) as bigint)) AS (1 - CAST(1 AS SMALLINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -210,7 +221,7 @@ Project [(cast(1 as double) - cast(cast(1 as smallint) as double)) AS (1 - CAST( -- !query SELECT '1' - cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as double) - cast(cast(1 as int) as double)) AS (1 - CAST(1 AS INT))#x] +Project [(cast(1 as bigint) - cast(cast(1 as int) as bigint)) AS (1 - CAST(1 AS INT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -221,7 +232,7 @@ Project [(cast(1 as double) - cast(cast(1 as int) as double)) AS (1 - CAST(1 AS -- !query SELECT '1' - cast(1 as bigint) FROM t -- !query analysis -Project [(cast(1 as double) - cast(cast(1 as bigint) as double)) AS (1 - CAST(1 AS BIGINT))#x] +Project [(cast(1 as bigint) - cast(1 as bigint)) AS (1 - CAST(1 AS BIGINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -265,12 +276,23 @@ Project [(cast(1 as double) - cast(cast(1 as decimal(10,0)) as double)) AS (1 - -- !query SELECT '1' - '1' FROM t -- !query analysis -Project [(cast(1 as double) - cast(1 as double)) AS (1 - 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' - '1'" + } ] +} -- !query @@ -278,11 +300,11 @@ SELECT '1' - cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -300,11 +322,11 @@ SELECT '1' - cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -320,25 +342,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(1 - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "'1' - cast('2017-12-11 09:30:00.0' as timestamp)" - } ] -} +Project [(cast(1 as timestamp) - cast(2017-12-11 09:30:00.0 as timestamp)) AS (1 - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -355,7 +364,7 @@ Project [(cast(1 as date) - cast(2017-12-11 09:30:00 as date)) AS (1 - CAST(2017 -- !query SELECT '1' * cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as double) * cast(cast(1 as tinyint) as double)) AS (1 * CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) * cast(cast(1 as tinyint) as bigint)) AS (1 * CAST(1 AS TINYINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -366,7 +375,7 @@ Project [(cast(1 as double) * cast(cast(1 as tinyint) as double)) AS (1 * CAST(1 -- !query SELECT '1' * cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as double) * cast(cast(1 as smallint) as double)) AS (1 * CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) * cast(cast(1 as smallint) as bigint)) AS (1 * CAST(1 AS SMALLINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -377,7 +386,7 @@ Project [(cast(1 as double) * cast(cast(1 as smallint) as double)) AS (1 * CAST( -- !query SELECT '1' * cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as double) * cast(cast(1 as int) as double)) AS (1 * CAST(1 AS INT))#x] +Project [(cast(1 as bigint) * cast(cast(1 as int) as bigint)) AS (1 * CAST(1 AS INT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -388,7 +397,7 @@ Project [(cast(1 as double) * cast(cast(1 as int) as double)) AS (1 * CAST(1 AS -- !query SELECT '1' * cast(1 as bigint) FROM t -- !query analysis -Project [(cast(1 as double) * cast(cast(1 as bigint) as double)) AS (1 * CAST(1 AS BIGINT))#x] +Project [(cast(1 as bigint) * cast(1 as bigint)) AS (1 * CAST(1 AS BIGINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -432,12 +441,23 @@ Project [(cast(1 as double) * cast(cast(1 as decimal(10,0)) as double)) AS (1 * -- !query SELECT '1' * '1' FROM t -- !query analysis -Project [(cast(1 as double) * cast(1 as double)) AS (1 * 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' * '1'" + } ] +} -- !query @@ -445,11 +465,11 @@ SELECT '1' * cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -467,11 +487,11 @@ SELECT '1' * cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -489,11 +509,11 @@ SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -511,11 +531,11 @@ SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -531,7 +551,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' / cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as double) / cast(cast(1 as tinyint) as double)) AS (1 / CAST(1 AS TINYINT))#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as tinyint) as bigint) as double)) AS (1 / CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -542,7 +562,7 @@ Project [(cast(1 as double) / cast(cast(1 as tinyint) as double)) AS (1 / CAST(1 -- !query SELECT '1' / cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as double) / cast(cast(1 as smallint) as double)) AS (1 / CAST(1 AS SMALLINT))#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as smallint) as bigint) as double)) AS (1 / CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -553,7 +573,7 @@ Project [(cast(1 as double) / cast(cast(1 as smallint) as double)) AS (1 / CAST( -- !query SELECT '1' / cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as double) / cast(cast(1 as int) as double)) AS (1 / CAST(1 AS INT))#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(cast(1 as int) as bigint) as double)) AS (1 / CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -564,7 +584,7 @@ Project [(cast(1 as double) / cast(cast(1 as int) as double)) AS (1 / CAST(1 AS -- !query SELECT '1' / cast(1 as bigint) FROM t -- !query analysis -Project [(cast(1 as double) / cast(cast(1 as bigint) as double)) AS (1 / CAST(1 AS BIGINT))#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as bigint) as double)) AS (1 / CAST(1 AS BIGINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -608,12 +628,23 @@ Project [(cast(1 as double) / cast(cast(1 as decimal(10,0)) as double)) AS (1 / -- !query SELECT '1' / '1' FROM t -- !query analysis -Project [(cast(1 as double) / cast(1 as double)) AS (1 / 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' / '1'" + } ] +} -- !query @@ -621,11 +652,11 @@ SELECT '1' / cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -643,11 +674,11 @@ SELECT '1' / cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -665,11 +696,11 @@ SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -687,11 +718,11 @@ SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -707,7 +738,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' % cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as double) % cast(cast(1 as tinyint) as double)) AS (1 % CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) % cast(cast(1 as tinyint) as bigint)) AS (1 % CAST(1 AS TINYINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -718,7 +749,7 @@ Project [(cast(1 as double) % cast(cast(1 as tinyint) as double)) AS (1 % CAST(1 -- !query SELECT '1' % cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as double) % cast(cast(1 as smallint) as double)) AS (1 % CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) % cast(cast(1 as smallint) as bigint)) AS (1 % CAST(1 AS SMALLINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -729,7 +760,7 @@ Project [(cast(1 as double) % cast(cast(1 as smallint) as double)) AS (1 % CAST( -- !query SELECT '1' % cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as double) % cast(cast(1 as int) as double)) AS (1 % CAST(1 AS INT))#x] +Project [(cast(1 as bigint) % cast(cast(1 as int) as bigint)) AS (1 % CAST(1 AS INT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -740,7 +771,7 @@ Project [(cast(1 as double) % cast(cast(1 as int) as double)) AS (1 % CAST(1 AS -- !query SELECT '1' % cast(1 as bigint) FROM t -- !query analysis -Project [(cast(1 as double) % cast(cast(1 as bigint) as double)) AS (1 % CAST(1 AS BIGINT))#x] +Project [(cast(1 as bigint) % cast(1 as bigint)) AS (1 % CAST(1 AS BIGINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -784,12 +815,23 @@ Project [(cast(1 as double) % cast(cast(1 as decimal(10,0)) as double)) AS (1 % -- !query SELECT '1' % '1' FROM t -- !query analysis -Project [(cast(1 as double) % cast(1 as double)) AS (1 % 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' % '1'" + } ] +} -- !query @@ -797,11 +839,11 @@ SELECT '1' % cast('1' as binary) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -819,11 +861,11 @@ SELECT '1' % cast(1 as boolean) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -841,11 +883,11 @@ SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -863,11 +905,11 @@ SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -883,7 +925,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT pmod('1', cast(1 as tinyint)) FROM t -- !query analysis -Project [pmod(cast(1 as double), cast(cast(1 as tinyint) as double)) AS pmod(1, CAST(1 AS TINYINT))#x] +Project [pmod(cast(1 as bigint), cast(cast(1 as tinyint) as bigint)) AS pmod(1, CAST(1 AS TINYINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -894,7 +936,7 @@ Project [pmod(cast(1 as double), cast(cast(1 as tinyint) as double)) AS pmod(1, -- !query SELECT pmod('1', cast(1 as smallint)) FROM t -- !query analysis -Project [pmod(cast(1 as double), cast(cast(1 as smallint) as double)) AS pmod(1, CAST(1 AS SMALLINT))#x] +Project [pmod(cast(1 as bigint), cast(cast(1 as smallint) as bigint)) AS pmod(1, CAST(1 AS SMALLINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -905,7 +947,7 @@ Project [pmod(cast(1 as double), cast(cast(1 as smallint) as double)) AS pmod(1, -- !query SELECT pmod('1', cast(1 as int)) FROM t -- !query analysis -Project [pmod(cast(1 as double), cast(cast(1 as int) as double)) AS pmod(1, CAST(1 AS INT))#x] +Project [pmod(cast(1 as bigint), cast(cast(1 as int) as bigint)) AS pmod(1, CAST(1 AS INT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -916,7 +958,7 @@ Project [pmod(cast(1 as double), cast(cast(1 as int) as double)) AS pmod(1, CAST -- !query SELECT pmod('1', cast(1 as bigint)) FROM t -- !query analysis -Project [pmod(cast(1 as double), cast(cast(1 as bigint) as double)) AS pmod(1, CAST(1 AS BIGINT))#x] +Project [pmod(cast(1 as bigint), cast(1 as bigint)) AS pmod(1, CAST(1 AS BIGINT))#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -960,12 +1002,23 @@ Project [pmod(cast(1 as double), cast(cast(1 as decimal(10,0)) as double)) AS pm -- !query SELECT pmod('1', '1') FROM t -- !query analysis -Project [pmod(cast(1 as double), cast(1 as double)) AS pmod(1, 1)#x] -+- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "pmod('1', '1')" + } ] +} -- !query @@ -973,11 +1026,11 @@ SELECT pmod('1', cast('1' as binary)) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -995,11 +1048,11 @@ SELECT pmod('1', cast(1 as boolean)) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -1017,11 +1070,11 @@ SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -1039,11 +1092,11 @@ SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -1059,7 +1112,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) + '1' FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) + cast(1 as double)) AS (CAST(1 AS TINYINT) + 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) + cast(1 as bigint)) AS (CAST(1 AS TINYINT) + 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1070,7 +1123,7 @@ Project [(cast(cast(1 as tinyint) as double) + cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as smallint) + '1' FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) + cast(1 as double)) AS (CAST(1 AS SMALLINT) + 1)#x] +Project [(cast(cast(1 as smallint) as bigint) + cast(1 as bigint)) AS (CAST(1 AS SMALLINT) + 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1081,7 +1134,7 @@ Project [(cast(cast(1 as smallint) as double) + cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as int) + '1' FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) + cast(1 as double)) AS (CAST(1 AS INT) + 1)#x] +Project [(cast(cast(1 as int) as bigint) + cast(1 as bigint)) AS (CAST(1 AS INT) + 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1092,7 +1145,7 @@ Project [(cast(cast(1 as int) as double) + cast(1 as double)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as bigint) + '1' FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) + cast(1 as double)) AS (CAST(1 AS BIGINT) + 1)#x] +Project [(cast(1 as bigint) + cast(1 as bigint)) AS (CAST(1 AS BIGINT) + 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1138,11 +1191,11 @@ SELECT cast('1' as binary) + '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"" }, "queryContext" : [ { @@ -1160,11 +1213,11 @@ SELECT cast(1 as boolean) + '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"" }, "queryContext" : [ { @@ -1182,11 +1235,11 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"" }, "queryContext" : [ { @@ -1208,7 +1261,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" @@ -1226,7 +1279,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) - '1' FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) - cast(1 as double)) AS (CAST(1 AS TINYINT) - 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) - cast(1 as bigint)) AS (CAST(1 AS TINYINT) - 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1237,7 +1290,7 @@ Project [(cast(cast(1 as tinyint) as double) - cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as smallint) - '1' FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) - cast(1 as double)) AS (CAST(1 AS SMALLINT) - 1)#x] +Project [(cast(cast(1 as smallint) as bigint) - cast(1 as bigint)) AS (CAST(1 AS SMALLINT) - 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1248,7 +1301,7 @@ Project [(cast(cast(1 as smallint) as double) - cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as int) - '1' FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) - cast(1 as double)) AS (CAST(1 AS INT) - 1)#x] +Project [(cast(cast(1 as int) as bigint) - cast(1 as bigint)) AS (CAST(1 AS INT) - 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1259,7 +1312,7 @@ Project [(cast(cast(1 as int) as double) - cast(1 as double)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as bigint) - '1' FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) - cast(1 as double)) AS (CAST(1 AS BIGINT) - 1)#x] +Project [(cast(1 as bigint) - cast(1 as bigint)) AS (CAST(1 AS BIGINT) - 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1305,11 +1358,11 @@ SELECT cast('1' as binary) - '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"" }, "queryContext" : [ { @@ -1327,11 +1380,11 @@ SELECT cast(1 as boolean) - '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"" }, "queryContext" : [ { @@ -1347,55 +1400,29 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - 1)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - '1'" - } ] -} +Project [(cast(2017-12-11 09:30:00.0 as timestamp) - cast(1 as timestamp)) AS (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - 1)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "cast('2017-12-11 09:30:00' as date) - '1'" - } ] -} +Project [(cast(2017-12-11 09:30:00 as date) - cast(1 as date)) AS (CAST(2017-12-11 09:30:00 AS DATE) - 1)#x] ++- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as tinyint) * '1' FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) * cast(1 as double)) AS (CAST(1 AS TINYINT) * 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) * cast(1 as bigint)) AS (CAST(1 AS TINYINT) * 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1406,7 +1433,7 @@ Project [(cast(cast(1 as tinyint) as double) * cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as smallint) * '1' FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) * cast(1 as double)) AS (CAST(1 AS SMALLINT) * 1)#x] +Project [(cast(cast(1 as smallint) as bigint) * cast(1 as bigint)) AS (CAST(1 AS SMALLINT) * 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1417,7 +1444,7 @@ Project [(cast(cast(1 as smallint) as double) * cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as int) * '1' FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) * cast(1 as double)) AS (CAST(1 AS INT) * 1)#x] +Project [(cast(cast(1 as int) as bigint) * cast(1 as bigint)) AS (CAST(1 AS INT) * 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1428,7 +1455,7 @@ Project [(cast(cast(1 as int) as double) * cast(1 as double)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as bigint) * '1' FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) * cast(1 as double)) AS (CAST(1 AS BIGINT) * 1)#x] +Project [(cast(1 as bigint) * cast(1 as bigint)) AS (CAST(1 AS BIGINT) * 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1474,11 +1501,11 @@ SELECT cast('1' as binary) * '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"" }, "queryContext" : [ { @@ -1496,11 +1523,11 @@ SELECT cast(1 as boolean) * '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"" }, "queryContext" : [ { @@ -1518,11 +1545,11 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"" }, "queryContext" : [ { @@ -1540,11 +1567,11 @@ SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"" }, "queryContext" : [ { @@ -1560,7 +1587,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) / '1' FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS TINYINT) / 1)#x] +Project [(cast(cast(cast(1 as tinyint) as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS TINYINT) / 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1571,7 +1598,7 @@ Project [(cast(cast(1 as tinyint) as double) / cast(cast(1 as double) as double) -- !query SELECT cast(1 as smallint) / '1' FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS SMALLINT) / 1)#x] +Project [(cast(cast(cast(1 as smallint) as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS SMALLINT) / 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1582,7 +1609,7 @@ Project [(cast(cast(1 as smallint) as double) / cast(cast(1 as double) as double -- !query SELECT cast(1 as int) / '1' FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS INT) / 1)#x] +Project [(cast(cast(cast(1 as int) as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS INT) / 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1593,7 +1620,7 @@ Project [(cast(cast(1 as int) as double) / cast(cast(1 as double) as double)) AS -- !query SELECT cast(1 as bigint) / '1' FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS BIGINT) / 1)#x] +Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as bigint) as double)) AS (CAST(1 AS BIGINT) / 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1604,7 +1631,7 @@ Project [(cast(cast(1 as bigint) as double) / cast(cast(1 as double) as double)) -- !query SELECT cast(1 as float) / '1' FROM t -- !query analysis -Project [(cast(cast(1 as float) as double) / cast(cast(1 as double) as double)) AS (CAST(1 AS FLOAT) / 1)#x] +Project [(cast(cast(1 as float) as double) / cast(1 as double)) AS (CAST(1 AS FLOAT) / 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1639,11 +1666,11 @@ SELECT cast('1' as binary) / '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"" }, "queryContext" : [ { @@ -1661,11 +1688,11 @@ SELECT cast(1 as boolean) / '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"" }, "queryContext" : [ { @@ -1683,11 +1710,11 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"" }, "queryContext" : [ { @@ -1705,11 +1732,11 @@ SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"" }, "queryContext" : [ { @@ -1725,7 +1752,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) % '1' FROM t -- !query analysis -Project [(cast(cast(1 as tinyint) as double) % cast(1 as double)) AS (CAST(1 AS TINYINT) % 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) % cast(1 as bigint)) AS (CAST(1 AS TINYINT) % 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1736,7 +1763,7 @@ Project [(cast(cast(1 as tinyint) as double) % cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as smallint) % '1' FROM t -- !query analysis -Project [(cast(cast(1 as smallint) as double) % cast(1 as double)) AS (CAST(1 AS SMALLINT) % 1)#x] +Project [(cast(cast(1 as smallint) as bigint) % cast(1 as bigint)) AS (CAST(1 AS SMALLINT) % 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1747,7 +1774,7 @@ Project [(cast(cast(1 as smallint) as double) % cast(1 as double)) AS (CAST(1 AS -- !query SELECT cast(1 as int) % '1' FROM t -- !query analysis -Project [(cast(cast(1 as int) as double) % cast(1 as double)) AS (CAST(1 AS INT) % 1)#x] +Project [(cast(cast(1 as int) as bigint) % cast(1 as bigint)) AS (CAST(1 AS INT) % 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1758,7 +1785,7 @@ Project [(cast(cast(1 as int) as double) % cast(1 as double)) AS (CAST(1 AS INT) -- !query SELECT cast(1 as bigint) % '1' FROM t -- !query analysis -Project [(cast(cast(1 as bigint) as double) % cast(1 as double)) AS (CAST(1 AS BIGINT) % 1)#x] +Project [(cast(1 as bigint) % cast(1 as bigint)) AS (CAST(1 AS BIGINT) % 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1804,11 +1831,11 @@ SELECT cast('1' as binary) % '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"" }, "queryContext" : [ { @@ -1826,11 +1853,11 @@ SELECT cast(1 as boolean) % '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"" }, "queryContext" : [ { @@ -1848,11 +1875,11 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"" }, "queryContext" : [ { @@ -1870,11 +1897,11 @@ SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"" }, "queryContext" : [ { @@ -1890,7 +1917,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT pmod(cast(1 as tinyint), '1') FROM t -- !query analysis -Project [pmod(cast(cast(1 as tinyint) as double), cast(1 as double)) AS pmod(CAST(1 AS TINYINT), 1)#x] +Project [pmod(cast(cast(1 as tinyint) as bigint), cast(1 as bigint)) AS pmod(CAST(1 AS TINYINT), 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1901,7 +1928,7 @@ Project [pmod(cast(cast(1 as tinyint) as double), cast(1 as double)) AS pmod(CAS -- !query SELECT pmod(cast(1 as smallint), '1') FROM t -- !query analysis -Project [pmod(cast(cast(1 as smallint) as double), cast(1 as double)) AS pmod(CAST(1 AS SMALLINT), 1)#x] +Project [pmod(cast(cast(1 as smallint) as bigint), cast(1 as bigint)) AS pmod(CAST(1 AS SMALLINT), 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1912,7 +1939,7 @@ Project [pmod(cast(cast(1 as smallint) as double), cast(1 as double)) AS pmod(CA -- !query SELECT pmod(cast(1 as int), '1') FROM t -- !query analysis -Project [pmod(cast(cast(1 as int) as double), cast(1 as double)) AS pmod(CAST(1 AS INT), 1)#x] +Project [pmod(cast(cast(1 as int) as bigint), cast(1 as bigint)) AS pmod(CAST(1 AS INT), 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1923,7 +1950,7 @@ Project [pmod(cast(cast(1 as int) as double), cast(1 as double)) AS pmod(CAST(1 -- !query SELECT pmod(cast(1 as bigint), '1') FROM t -- !query analysis -Project [pmod(cast(cast(1 as bigint) as double), cast(1 as double)) AS pmod(CAST(1 AS BIGINT), 1)#x] +Project [pmod(cast(1 as bigint), cast(1 as bigint)) AS pmod(CAST(1 AS BIGINT), 1)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -1969,11 +1996,11 @@ SELECT pmod(cast('1' as binary), '1') FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"" }, "queryContext" : [ { @@ -1991,11 +2018,11 @@ SELECT pmod(cast(1 as boolean), '1') FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"" }, "queryContext" : [ { @@ -2013,11 +2040,11 @@ SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"" }, "queryContext" : [ { @@ -2035,11 +2062,11 @@ SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" }, "queryContext" : [ { @@ -2055,7 +2082,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' = cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (1 = CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as tinyint) as bigint)) AS (1 = CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2066,7 +2093,7 @@ Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (1 = CAST(1 AS TINYINT))#x -- !query SELECT '1' = cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(1 as smallint)) AS (1 = CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as smallint) as bigint)) AS (1 = CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2077,7 +2104,7 @@ Project [(cast(1 as smallint) = cast(1 as smallint)) AS (1 = CAST(1 AS SMALLINT) -- !query SELECT '1' = cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) = cast(1 as int)) AS (1 = CAST(1 AS INT))#x] +Project [(cast(1 as bigint) = cast(cast(1 as int) as bigint)) AS (1 = CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2099,7 +2126,7 @@ Project [(cast(1 as bigint) = cast(1 as bigint)) AS (1 = CAST(1 AS BIGINT))#x] -- !query SELECT '1' = cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) = cast(1 as float)) AS (1 = CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) = cast(cast(1 as float) as double)) AS (1 = CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2121,7 +2148,7 @@ Project [(cast(1 as double) = cast(1 as double)) AS (1 = CAST(1 AS DOUBLE))#x] -- !query SELECT '1' = cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (1 = CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (1 = CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2187,7 +2214,7 @@ Project [(cast(1 as date) = cast(2017-12-11 09:30:00 as date)) AS (1 = CAST(2017 -- !query SELECT cast(1 as tinyint) = '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (CAST(1 AS TINYINT) = 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) = cast(1 as bigint)) AS (CAST(1 AS TINYINT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2198,7 +2225,7 @@ Project [(cast(1 as tinyint) = cast(1 as tinyint)) AS (CAST(1 AS TINYINT) = 1)#x -- !query SELECT cast(1 as smallint) = '1' FROM t -- !query analysis -Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 1)#x] +Project [(cast(cast(1 as smallint) as bigint) = cast(1 as bigint)) AS (CAST(1 AS SMALLINT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2209,7 +2236,7 @@ Project [(cast(1 as smallint) = cast(1 as smallint)) AS (CAST(1 AS SMALLINT) = 1 -- !query SELECT cast(1 as int) = '1' FROM t -- !query analysis -Project [(cast(1 as int) = cast(1 as int)) AS (CAST(1 AS INT) = 1)#x] +Project [(cast(cast(1 as int) as bigint) = cast(1 as bigint)) AS (CAST(1 AS INT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2231,7 +2258,7 @@ Project [(cast(1 as bigint) = cast(1 as bigint)) AS (CAST(1 AS BIGINT) = 1)#x] -- !query SELECT cast(1 as float) = '1' FROM t -- !query analysis -Project [(cast(1 as float) = cast(1 as float)) AS (CAST(1 AS FLOAT) = 1)#x] +Project [(cast(cast(1 as float) as double) = cast(1 as double)) AS (CAST(1 AS FLOAT) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2253,7 +2280,7 @@ Project [(cast(1 as double) = cast(1 as double)) AS (CAST(1 AS DOUBLE) = 1)#x] -- !query SELECT cast(1 as decimal(10, 0)) = '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) = 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) = cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) = 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2308,7 +2335,7 @@ Project [(cast(2017-12-11 09:30:00 as date) = cast(1 as date)) AS (CAST(2017-12- -- !query SELECT '1' <=> cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) <=> cast(1 as tinyint)) AS (1 <=> CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) <=> cast(cast(1 as tinyint) as bigint)) AS (1 <=> CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2319,7 +2346,7 @@ Project [(cast(1 as tinyint) <=> cast(1 as tinyint)) AS (1 <=> CAST(1 AS TINYINT -- !query SELECT '1' <=> cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) <=> cast(1 as smallint)) AS (1 <=> CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) <=> cast(cast(1 as smallint) as bigint)) AS (1 <=> CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2330,7 +2357,7 @@ Project [(cast(1 as smallint) <=> cast(1 as smallint)) AS (1 <=> CAST(1 AS SMALL -- !query SELECT '1' <=> cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) <=> cast(1 as int)) AS (1 <=> CAST(1 AS INT))#x] +Project [(cast(1 as bigint) <=> cast(cast(1 as int) as bigint)) AS (1 <=> CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2352,7 +2379,7 @@ Project [(cast(1 as bigint) <=> cast(1 as bigint)) AS (1 <=> CAST(1 AS BIGINT))# -- !query SELECT '1' <=> cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) <=> cast(1 as float)) AS (1 <=> CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) <=> cast(cast(1 as float) as double)) AS (1 <=> CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2374,7 +2401,7 @@ Project [(cast(1 as double) <=> cast(1 as double)) AS (1 <=> CAST(1 AS DOUBLE))# -- !query SELECT '1' <=> cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (1 <=> CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) <=> cast(cast(1 as decimal(10,0)) as double)) AS (1 <=> CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2440,7 +2467,7 @@ Project [(cast(1 as date) <=> cast(2017-12-11 09:30:00 as date)) AS (1 <=> CAST( -- !query SELECT cast(1 as tinyint) <=> '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) <=> cast(1 as tinyint)) AS (CAST(1 AS TINYINT) <=> 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) <=> cast(1 as bigint)) AS (CAST(1 AS TINYINT) <=> 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2451,7 +2478,7 @@ Project [(cast(1 as tinyint) <=> cast(1 as tinyint)) AS (CAST(1 AS TINYINT) <=> -- !query SELECT cast(1 as smallint) <=> '1' FROM t -- !query analysis -Project [(cast(1 as smallint) <=> cast(1 as smallint)) AS (CAST(1 AS SMALLINT) <=> 1)#x] +Project [(cast(cast(1 as smallint) as bigint) <=> cast(1 as bigint)) AS (CAST(1 AS SMALLINT) <=> 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2462,7 +2489,7 @@ Project [(cast(1 as smallint) <=> cast(1 as smallint)) AS (CAST(1 AS SMALLINT) < -- !query SELECT cast(1 as int) <=> '1' FROM t -- !query analysis -Project [(cast(1 as int) <=> cast(1 as int)) AS (CAST(1 AS INT) <=> 1)#x] +Project [(cast(cast(1 as int) as bigint) <=> cast(1 as bigint)) AS (CAST(1 AS INT) <=> 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2484,7 +2511,7 @@ Project [(cast(1 as bigint) <=> cast(1 as bigint)) AS (CAST(1 AS BIGINT) <=> 1)# -- !query SELECT cast(1 as float) <=> '1' FROM t -- !query analysis -Project [(cast(1 as float) <=> cast(1 as float)) AS (CAST(1 AS FLOAT) <=> 1)#x] +Project [(cast(cast(1 as float) as double) <=> cast(1 as double)) AS (CAST(1 AS FLOAT) <=> 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2506,7 +2533,7 @@ Project [(cast(1 as double) <=> cast(1 as double)) AS (CAST(1 AS DOUBLE) <=> 1)# -- !query SELECT cast(1 as decimal(10, 0)) <=> '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <=> cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <=> 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <=> cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) <=> 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2561,7 +2588,7 @@ Project [(cast(2017-12-11 09:30:00 as date) <=> cast(1 as date)) AS (CAST(2017-1 -- !query SELECT '1' < cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) < cast(1 as tinyint)) AS (1 < CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) < cast(cast(1 as tinyint) as bigint)) AS (1 < CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2572,7 +2599,7 @@ Project [(cast(1 as tinyint) < cast(1 as tinyint)) AS (1 < CAST(1 AS TINYINT))#x -- !query SELECT '1' < cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) < cast(1 as smallint)) AS (1 < CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) < cast(cast(1 as smallint) as bigint)) AS (1 < CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2583,7 +2610,7 @@ Project [(cast(1 as smallint) < cast(1 as smallint)) AS (1 < CAST(1 AS SMALLINT) -- !query SELECT '1' < cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) < cast(1 as int)) AS (1 < CAST(1 AS INT))#x] +Project [(cast(1 as bigint) < cast(cast(1 as int) as bigint)) AS (1 < CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2605,7 +2632,7 @@ Project [(cast(1 as bigint) < cast(1 as bigint)) AS (1 < CAST(1 AS BIGINT))#x] -- !query SELECT '1' < cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) < cast(1 as float)) AS (1 < CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) < cast(cast(1 as float) as double)) AS (1 < CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2627,7 +2654,7 @@ Project [(cast(1 as double) < cast(1 as double)) AS (1 < CAST(1 AS DOUBLE))#x] -- !query SELECT '1' < cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (1 < CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) < cast(cast(1 as decimal(10,0)) as double)) AS (1 < CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2693,7 +2720,7 @@ Project [(cast(1 as date) < cast(2017-12-11 09:30:00 as date)) AS (1 < CAST(2017 -- !query SELECT '1' <= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) <= cast(1 as tinyint)) AS (1 <= CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) <= cast(cast(1 as tinyint) as bigint)) AS (1 <= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2704,7 +2731,7 @@ Project [(cast(1 as tinyint) <= cast(1 as tinyint)) AS (1 <= CAST(1 AS TINYINT)) -- !query SELECT '1' <= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) <= cast(1 as smallint)) AS (1 <= CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) <= cast(cast(1 as smallint) as bigint)) AS (1 <= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2715,7 +2742,7 @@ Project [(cast(1 as smallint) <= cast(1 as smallint)) AS (1 <= CAST(1 AS SMALLIN -- !query SELECT '1' <= cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) <= cast(1 as int)) AS (1 <= CAST(1 AS INT))#x] +Project [(cast(1 as bigint) <= cast(cast(1 as int) as bigint)) AS (1 <= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2737,7 +2764,7 @@ Project [(cast(1 as bigint) <= cast(1 as bigint)) AS (1 <= CAST(1 AS BIGINT))#x] -- !query SELECT '1' <= cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) <= cast(1 as float)) AS (1 <= CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) <= cast(cast(1 as float) as double)) AS (1 <= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2759,7 +2786,7 @@ Project [(cast(1 as double) <= cast(1 as double)) AS (1 <= CAST(1 AS DOUBLE))#x] -- !query SELECT '1' <= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (1 <= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) <= cast(cast(1 as decimal(10,0)) as double)) AS (1 <= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2825,7 +2852,7 @@ Project [(cast(1 as date) <= cast(2017-12-11 09:30:00 as date)) AS (1 <= CAST(20 -- !query SELECT '1' > cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) > cast(1 as tinyint)) AS (1 > CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) > cast(cast(1 as tinyint) as bigint)) AS (1 > CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2836,7 +2863,7 @@ Project [(cast(1 as tinyint) > cast(1 as tinyint)) AS (1 > CAST(1 AS TINYINT))#x -- !query SELECT '1' > cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) > cast(1 as smallint)) AS (1 > CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) > cast(cast(1 as smallint) as bigint)) AS (1 > CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2847,7 +2874,7 @@ Project [(cast(1 as smallint) > cast(1 as smallint)) AS (1 > CAST(1 AS SMALLINT) -- !query SELECT '1' > cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) > cast(1 as int)) AS (1 > CAST(1 AS INT))#x] +Project [(cast(1 as bigint) > cast(cast(1 as int) as bigint)) AS (1 > CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2869,7 +2896,7 @@ Project [(cast(1 as bigint) > cast(1 as bigint)) AS (1 > CAST(1 AS BIGINT))#x] -- !query SELECT '1' > cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) > cast(1 as float)) AS (1 > CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) > cast(cast(1 as float) as double)) AS (1 > CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2891,7 +2918,7 @@ Project [(cast(1 as double) > cast(1 as double)) AS (1 > CAST(1 AS DOUBLE))#x] -- !query SELECT '1' > cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (1 > CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) > cast(cast(1 as decimal(10,0)) as double)) AS (1 > CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2957,7 +2984,7 @@ Project [(cast(1 as date) > cast(2017-12-11 09:30:00 as date)) AS (1 > CAST(2017 -- !query SELECT '1' >= cast(1 as tinyint) FROM t -- !query analysis -Project [(cast(1 as tinyint) >= cast(1 as tinyint)) AS (1 >= CAST(1 AS TINYINT))#x] +Project [(cast(1 as bigint) >= cast(cast(1 as tinyint) as bigint)) AS (1 >= CAST(1 AS TINYINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2968,7 +2995,7 @@ Project [(cast(1 as tinyint) >= cast(1 as tinyint)) AS (1 >= CAST(1 AS TINYINT)) -- !query SELECT '1' >= cast(1 as smallint) FROM t -- !query analysis -Project [(cast(1 as smallint) >= cast(1 as smallint)) AS (1 >= CAST(1 AS SMALLINT))#x] +Project [(cast(1 as bigint) >= cast(cast(1 as smallint) as bigint)) AS (1 >= CAST(1 AS SMALLINT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -2979,7 +3006,7 @@ Project [(cast(1 as smallint) >= cast(1 as smallint)) AS (1 >= CAST(1 AS SMALLIN -- !query SELECT '1' >= cast(1 as int) FROM t -- !query analysis -Project [(cast(1 as int) >= cast(1 as int)) AS (1 >= CAST(1 AS INT))#x] +Project [(cast(1 as bigint) >= cast(cast(1 as int) as bigint)) AS (1 >= CAST(1 AS INT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3001,7 +3028,7 @@ Project [(cast(1 as bigint) >= cast(1 as bigint)) AS (1 >= CAST(1 AS BIGINT))#x] -- !query SELECT '1' >= cast(1 as float) FROM t -- !query analysis -Project [(cast(1 as float) >= cast(1 as float)) AS (1 >= CAST(1 AS FLOAT))#x] +Project [(cast(1 as double) >= cast(cast(1 as float) as double)) AS (1 >= CAST(1 AS FLOAT))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3023,7 +3050,7 @@ Project [(cast(1 as double) >= cast(1 as double)) AS (1 >= CAST(1 AS DOUBLE))#x] -- !query SELECT '1' >= cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (1 >= CAST(1 AS DECIMAL(10,0)))#x] +Project [(cast(1 as double) >= cast(cast(1 as decimal(10,0)) as double)) AS (1 >= CAST(1 AS DECIMAL(10,0)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3089,7 +3116,7 @@ Project [(cast(1 as date) >= cast(2017-12-11 09:30:00 as date)) AS (1 >= CAST(20 -- !query SELECT '1' <> cast(1 as tinyint) FROM t -- !query analysis -Project [NOT (cast(1 as tinyint) = cast(1 as tinyint)) AS (NOT (1 = CAST(1 AS TINYINT)))#x] +Project [NOT (cast(1 as bigint) = cast(cast(1 as tinyint) as bigint)) AS (NOT (1 = CAST(1 AS TINYINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3100,7 +3127,7 @@ Project [NOT (cast(1 as tinyint) = cast(1 as tinyint)) AS (NOT (1 = CAST(1 AS TI -- !query SELECT '1' <> cast(1 as smallint) FROM t -- !query analysis -Project [NOT (cast(1 as smallint) = cast(1 as smallint)) AS (NOT (1 = CAST(1 AS SMALLINT)))#x] +Project [NOT (cast(1 as bigint) = cast(cast(1 as smallint) as bigint)) AS (NOT (1 = CAST(1 AS SMALLINT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3111,7 +3138,7 @@ Project [NOT (cast(1 as smallint) = cast(1 as smallint)) AS (NOT (1 = CAST(1 AS -- !query SELECT '1' <> cast(1 as int) FROM t -- !query analysis -Project [NOT (cast(1 as int) = cast(1 as int)) AS (NOT (1 = CAST(1 AS INT)))#x] +Project [NOT (cast(1 as bigint) = cast(cast(1 as int) as bigint)) AS (NOT (1 = CAST(1 AS INT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3133,7 +3160,7 @@ Project [NOT (cast(1 as bigint) = cast(1 as bigint)) AS (NOT (1 = CAST(1 AS BIGI -- !query SELECT '1' <> cast(1 as float) FROM t -- !query analysis -Project [NOT (cast(1 as float) = cast(1 as float)) AS (NOT (1 = CAST(1 AS FLOAT)))#x] +Project [NOT (cast(1 as double) = cast(cast(1 as float) as double)) AS (NOT (1 = CAST(1 AS FLOAT)))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3155,7 +3182,7 @@ Project [NOT (cast(1 as double) = cast(1 as double)) AS (NOT (1 = CAST(1 AS DOUB -- !query SELECT '1' <> cast(1 as decimal(10, 0)) FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (1 = CAST(1 AS DECIMAL(10,0))))#x] +Project [NOT (cast(1 as double) = cast(cast(1 as decimal(10,0)) as double)) AS (NOT (1 = CAST(1 AS DECIMAL(10,0))))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3221,7 +3248,7 @@ Project [NOT (cast(1 as date) = cast(2017-12-11 09:30:00 as date)) AS (NOT (1 = -- !query SELECT cast(1 as tinyint) < '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) < cast(1 as tinyint)) AS (CAST(1 AS TINYINT) < 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) < cast(1 as bigint)) AS (CAST(1 AS TINYINT) < 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3232,7 +3259,7 @@ Project [(cast(1 as tinyint) < cast(1 as tinyint)) AS (CAST(1 AS TINYINT) < 1)#x -- !query SELECT cast(1 as smallint) < '1' FROM t -- !query analysis -Project [(cast(1 as smallint) < cast(1 as smallint)) AS (CAST(1 AS SMALLINT) < 1)#x] +Project [(cast(cast(1 as smallint) as bigint) < cast(1 as bigint)) AS (CAST(1 AS SMALLINT) < 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3243,7 +3270,7 @@ Project [(cast(1 as smallint) < cast(1 as smallint)) AS (CAST(1 AS SMALLINT) < 1 -- !query SELECT cast(1 as int) < '1' FROM t -- !query analysis -Project [(cast(1 as int) < cast(1 as int)) AS (CAST(1 AS INT) < 1)#x] +Project [(cast(cast(1 as int) as bigint) < cast(1 as bigint)) AS (CAST(1 AS INT) < 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3265,7 +3292,7 @@ Project [(cast(1 as bigint) < cast(1 as bigint)) AS (CAST(1 AS BIGINT) < 1)#x] -- !query SELECT cast(1 as float) < '1' FROM t -- !query analysis -Project [(cast(1 as float) < cast(1 as float)) AS (CAST(1 AS FLOAT) < 1)#x] +Project [(cast(cast(1 as float) as double) < cast(1 as double)) AS (CAST(1 AS FLOAT) < 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3287,7 +3314,7 @@ Project [(cast(1 as double) < cast(1 as double)) AS (CAST(1 AS DOUBLE) < 1)#x] -- !query SELECT cast(1 as decimal(10, 0)) < '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) < cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) < 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) < cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) < 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3353,7 +3380,7 @@ Project [(cast(2017-12-11 09:30:00 as date) < cast(1 as date)) AS (CAST(2017-12- -- !query SELECT cast(1 as tinyint) <= '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) <= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) <= 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) <= cast(1 as bigint)) AS (CAST(1 AS TINYINT) <= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3364,7 +3391,7 @@ Project [(cast(1 as tinyint) <= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) <= 1) -- !query SELECT cast(1 as smallint) <= '1' FROM t -- !query analysis -Project [(cast(1 as smallint) <= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) <= 1)#x] +Project [(cast(cast(1 as smallint) as bigint) <= cast(1 as bigint)) AS (CAST(1 AS SMALLINT) <= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3375,7 +3402,7 @@ Project [(cast(1 as smallint) <= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) <= -- !query SELECT cast(1 as int) <= '1' FROM t -- !query analysis -Project [(cast(1 as int) <= cast(1 as int)) AS (CAST(1 AS INT) <= 1)#x] +Project [(cast(cast(1 as int) as bigint) <= cast(1 as bigint)) AS (CAST(1 AS INT) <= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3397,7 +3424,7 @@ Project [(cast(1 as bigint) <= cast(1 as bigint)) AS (CAST(1 AS BIGINT) <= 1)#x] -- !query SELECT cast(1 as float) <= '1' FROM t -- !query analysis -Project [(cast(1 as float) <= cast(1 as float)) AS (CAST(1 AS FLOAT) <= 1)#x] +Project [(cast(cast(1 as float) as double) <= cast(1 as double)) AS (CAST(1 AS FLOAT) <= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3419,7 +3446,7 @@ Project [(cast(1 as double) <= cast(1 as double)) AS (CAST(1 AS DOUBLE) <= 1)#x] -- !query SELECT cast(1 as decimal(10, 0)) <= '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) <= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) <= 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) <= cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) <= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3485,7 +3512,7 @@ Project [(cast(2017-12-11 09:30:00 as date) <= cast(1 as date)) AS (CAST(2017-12 -- !query SELECT cast(1 as tinyint) > '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) > cast(1 as tinyint)) AS (CAST(1 AS TINYINT) > 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) > cast(1 as bigint)) AS (CAST(1 AS TINYINT) > 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3496,7 +3523,7 @@ Project [(cast(1 as tinyint) > cast(1 as tinyint)) AS (CAST(1 AS TINYINT) > 1)#x -- !query SELECT cast(1 as smallint) > '1' FROM t -- !query analysis -Project [(cast(1 as smallint) > cast(1 as smallint)) AS (CAST(1 AS SMALLINT) > 1)#x] +Project [(cast(cast(1 as smallint) as bigint) > cast(1 as bigint)) AS (CAST(1 AS SMALLINT) > 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3507,7 +3534,7 @@ Project [(cast(1 as smallint) > cast(1 as smallint)) AS (CAST(1 AS SMALLINT) > 1 -- !query SELECT cast(1 as int) > '1' FROM t -- !query analysis -Project [(cast(1 as int) > cast(1 as int)) AS (CAST(1 AS INT) > 1)#x] +Project [(cast(cast(1 as int) as bigint) > cast(1 as bigint)) AS (CAST(1 AS INT) > 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3529,7 +3556,7 @@ Project [(cast(1 as bigint) > cast(1 as bigint)) AS (CAST(1 AS BIGINT) > 1)#x] -- !query SELECT cast(1 as float) > '1' FROM t -- !query analysis -Project [(cast(1 as float) > cast(1 as float)) AS (CAST(1 AS FLOAT) > 1)#x] +Project [(cast(cast(1 as float) as double) > cast(1 as double)) AS (CAST(1 AS FLOAT) > 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3551,7 +3578,7 @@ Project [(cast(1 as double) > cast(1 as double)) AS (CAST(1 AS DOUBLE) > 1)#x] -- !query SELECT cast(1 as decimal(10, 0)) > '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) > cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) > 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) > cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) > 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3617,7 +3644,7 @@ Project [(cast(2017-12-11 09:30:00 as date) > cast(1 as date)) AS (CAST(2017-12- -- !query SELECT cast(1 as tinyint) >= '1' FROM t -- !query analysis -Project [(cast(1 as tinyint) >= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) >= 1)#x] +Project [(cast(cast(1 as tinyint) as bigint) >= cast(1 as bigint)) AS (CAST(1 AS TINYINT) >= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3628,7 +3655,7 @@ Project [(cast(1 as tinyint) >= cast(1 as tinyint)) AS (CAST(1 AS TINYINT) >= 1) -- !query SELECT cast(1 as smallint) >= '1' FROM t -- !query analysis -Project [(cast(1 as smallint) >= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) >= 1)#x] +Project [(cast(cast(1 as smallint) as bigint) >= cast(1 as bigint)) AS (CAST(1 AS SMALLINT) >= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3639,7 +3666,7 @@ Project [(cast(1 as smallint) >= cast(1 as smallint)) AS (CAST(1 AS SMALLINT) >= -- !query SELECT cast(1 as int) >= '1' FROM t -- !query analysis -Project [(cast(1 as int) >= cast(1 as int)) AS (CAST(1 AS INT) >= 1)#x] +Project [(cast(cast(1 as int) as bigint) >= cast(1 as bigint)) AS (CAST(1 AS INT) >= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3661,7 +3688,7 @@ Project [(cast(1 as bigint) >= cast(1 as bigint)) AS (CAST(1 AS BIGINT) >= 1)#x] -- !query SELECT cast(1 as float) >= '1' FROM t -- !query analysis -Project [(cast(1 as float) >= cast(1 as float)) AS (CAST(1 AS FLOAT) >= 1)#x] +Project [(cast(cast(1 as float) as double) >= cast(1 as double)) AS (CAST(1 AS FLOAT) >= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3683,7 +3710,7 @@ Project [(cast(1 as double) >= cast(1 as double)) AS (CAST(1 AS DOUBLE) >= 1)#x] -- !query SELECT cast(1 as decimal(10, 0)) >= '1' FROM t -- !query analysis -Project [(cast(1 as decimal(10,0)) >= cast(1 as decimal(10,0))) AS (CAST(1 AS DECIMAL(10,0)) >= 1)#x] +Project [(cast(cast(1 as decimal(10,0)) as double) >= cast(1 as double)) AS (CAST(1 AS DECIMAL(10,0)) >= 1)#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3749,7 +3776,7 @@ Project [(cast(2017-12-11 09:30:00 as date) >= cast(1 as date)) AS (CAST(2017-12 -- !query SELECT cast(1 as tinyint) <> '1' FROM t -- !query analysis -Project [NOT (cast(1 as tinyint) = cast(1 as tinyint)) AS (NOT (CAST(1 AS TINYINT) = 1))#x] +Project [NOT (cast(cast(1 as tinyint) as bigint) = cast(1 as bigint)) AS (NOT (CAST(1 AS TINYINT) = 1))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3760,7 +3787,7 @@ Project [NOT (cast(1 as tinyint) = cast(1 as tinyint)) AS (NOT (CAST(1 AS TINYIN -- !query SELECT cast(1 as smallint) <> '1' FROM t -- !query analysis -Project [NOT (cast(1 as smallint) = cast(1 as smallint)) AS (NOT (CAST(1 AS SMALLINT) = 1))#x] +Project [NOT (cast(cast(1 as smallint) as bigint) = cast(1 as bigint)) AS (NOT (CAST(1 AS SMALLINT) = 1))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3771,7 +3798,7 @@ Project [NOT (cast(1 as smallint) = cast(1 as smallint)) AS (NOT (CAST(1 AS SMAL -- !query SELECT cast(1 as int) <> '1' FROM t -- !query analysis -Project [NOT (cast(1 as int) = cast(1 as int)) AS (NOT (CAST(1 AS INT) = 1))#x] +Project [NOT (cast(cast(1 as int) as bigint) = cast(1 as bigint)) AS (NOT (CAST(1 AS INT) = 1))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3793,7 +3820,7 @@ Project [NOT (cast(1 as bigint) = cast(1 as bigint)) AS (NOT (CAST(1 AS BIGINT) -- !query SELECT cast(1 as float) <> '1' FROM t -- !query analysis -Project [NOT (cast(1 as float) = cast(1 as float)) AS (NOT (CAST(1 AS FLOAT) = 1))#x] +Project [NOT (cast(cast(1 as float) as double) = cast(1 as double)) AS (NOT (CAST(1 AS FLOAT) = 1))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] @@ -3815,7 +3842,7 @@ Project [NOT (cast(1 as double) = cast(1 as double)) AS (NOT (CAST(1 AS DOUBLE) -- !query SELECT cast(1 as decimal(10, 0)) <> '1' FROM t -- !query analysis -Project [NOT (cast(1 as decimal(10,0)) = cast(1 as decimal(10,0))) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 1))#x] +Project [NOT (cast(cast(1 as decimal(10,0)) as double) = cast(1 as double)) AS (NOT (CAST(1 AS DECIMAL(10,0)) = 1))#x] +- SubqueryAlias t +- View (`t`, [1#x]) +- Project [cast(1#x as int) AS 1#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out index 22e60d0606382..e57f803124ee3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -197,7 +197,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select to_timestamp(a) from t -- !query analysis -Project [to_timestamp(a#x, None, TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(a)#x] +Project [to_timestamp(a#x, None, TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(a)#x] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -208,7 +208,7 @@ Project [to_timestamp(a#x, None, TimestampType, Some(America/Los_Angeles), false -- !query select to_timestamp('2018-01-01', a) from t -- !query analysis -Project [to_timestamp(2018-01-01, Some(a#x), TimestampType, Some(America/Los_Angeles), false) AS to_timestamp(2018-01-01, a)#x] +Project [to_timestamp(2018-01-01, Some(a#x), TimestampType, Some(America/Los_Angeles), true) AS to_timestamp(2018-01-01, a)#x] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -219,7 +219,7 @@ Project [to_timestamp(2018-01-01, Some(a#x), TimestampType, Some(America/Los_Ang -- !query select to_unix_timestamp(a) from t -- !query analysis -Project [to_unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), false) AS to_unix_timestamp(a, yyyy-MM-dd HH:mm:ss)#xL] +Project [to_unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), true) AS to_unix_timestamp(a, yyyy-MM-dd HH:mm:ss)#xL] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -230,7 +230,7 @@ Project [to_unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), -- !query select to_unix_timestamp('2018-01-01', a) from t -- !query analysis -Project [to_unix_timestamp(2018-01-01, a#x, Some(America/Los_Angeles), false) AS to_unix_timestamp(2018-01-01, a)#xL] +Project [to_unix_timestamp(2018-01-01, a#x, Some(America/Los_Angeles), true) AS to_unix_timestamp(2018-01-01, a)#xL] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -241,7 +241,7 @@ Project [to_unix_timestamp(2018-01-01, a#x, Some(America/Los_Angeles), false) AS -- !query select unix_timestamp(a) from t -- !query analysis -Project [unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), false) AS unix_timestamp(a, yyyy-MM-dd HH:mm:ss)#xL] +Project [unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), true) AS unix_timestamp(a, yyyy-MM-dd HH:mm:ss)#xL] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -252,7 +252,7 @@ Project [unix_timestamp(a#x, yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles), fal -- !query select unix_timestamp('2018-01-01', a) from t -- !query analysis -Project [unix_timestamp(2018-01-01, a#x, Some(America/Los_Angeles), false) AS unix_timestamp(2018-01-01, a)#xL] +Project [unix_timestamp(2018-01-01, a#x, Some(America/Los_Angeles), true) AS unix_timestamp(2018-01-01, a)#xL] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -285,7 +285,7 @@ Project [from_unixtime(cast(2018-01-01 as bigint), a#x, Some(America/Los_Angeles -- !query select next_day(a, 'MO') from t -- !query analysis -Project [next_day(cast(a#x as date), MO, false) AS next_day(a, MO)#x] +Project [next_day(cast(a#x as date), MO, true) AS next_day(a, MO)#x] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] @@ -296,7 +296,7 @@ Project [next_day(cast(a#x as date), MO, false) AS next_day(a, MO)#x] -- !query select next_day('2018-01-01', a) from t -- !query analysis -Project [next_day(cast(2018-01-01 as date), a#x, false) AS next_day(2018-01-01, a)#x] +Project [next_day(cast(2018-01-01 as date), a#x, true) AS next_day(2018-01-01, a)#x] +- SubqueryAlias t +- View (`t`, [a#x]) +- Project [cast(a#x as string) AS a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out index 029ec4abb6faf..e855cdc14a921 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -91,19 +91,20 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS TINYINT)#x as float) AS CAST(1 AS TINYINT)#x] + :- Project [cast(CAST(1 AS TINYINT)#x as double) AS CAST(1 AS TINYINT)#x] : +- Project [cast(1 as tinyint) AS CAST(1 AS TINYINT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS FLOAT)#x as double) AS CAST(2 AS FLOAT)#x] + +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -151,19 +152,20 @@ SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS TINYINT)#x as string) AS CAST(1 AS TINYINT)#x] + :- Project [cast(CAST(1 AS TINYINT)#x as bigint) AS CAST(1 AS TINYINT)#xL] : +- Project [cast(1 as tinyint) AS CAST(1 AS TINYINT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as bigint) AS CAST(2 AS STRING)#xL] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -350,19 +352,20 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS SMALLINT)#x as float) AS CAST(1 AS SMALLINT)#x] + :- Project [cast(CAST(1 AS SMALLINT)#x as double) AS CAST(1 AS SMALLINT)#x] : +- Project [cast(1 as smallint) AS CAST(1 AS SMALLINT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS FLOAT)#x as double) AS CAST(2 AS FLOAT)#x] + +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -410,19 +413,20 @@ SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS SMALLINT)#x as string) AS CAST(1 AS SMALLINT)#x] + :- Project [cast(CAST(1 AS SMALLINT)#x as bigint) AS CAST(1 AS SMALLINT)#xL] : +- Project [cast(1 as smallint) AS CAST(1 AS SMALLINT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as bigint) AS CAST(2 AS STRING)#xL] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -609,19 +613,20 @@ SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS INT)#x as float) AS CAST(1 AS INT)#x] + :- Project [cast(CAST(1 AS INT)#x as double) AS CAST(1 AS INT)#x] : +- Project [cast(1 as int) AS CAST(1 AS INT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS FLOAT)#x as double) AS CAST(2 AS FLOAT)#x] + +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -669,19 +674,20 @@ SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS INT)#x as string) AS CAST(1 AS INT)#x] + :- Project [cast(CAST(1 AS INT)#x as bigint) AS CAST(1 AS INT)#xL] : +- Project [cast(1 as int) AS CAST(1 AS INT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as bigint) AS CAST(2 AS STRING)#xL] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -868,19 +874,20 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS BIGINT)#xL as float) AS CAST(1 AS BIGINT)#x] + :- Project [cast(CAST(1 AS BIGINT)#xL as double) AS CAST(1 AS BIGINT)#x] : +- Project [cast(1 as bigint) AS CAST(1 AS BIGINT)#xL] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS FLOAT)#x as double) AS CAST(2 AS FLOAT)#x] + +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -929,19 +936,19 @@ SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS BIGINT)#xL as string) AS CAST(1 AS BIGINT)#x] - : +- Project [cast(1 as bigint) AS CAST(1 AS BIGINT)#xL] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(1 as bigint) AS CAST(1 AS BIGINT)#xL] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as bigint) AS CAST(2 AS STRING)#xL] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1049,13 +1056,14 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS TINYINT)#x as float) AS CAST(2 AS TINYINT)#x] + :- Project [cast(CAST(1 AS FLOAT)#x as double) AS CAST(1 AS FLOAT)#x] + : +- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS TINYINT)#x as double) AS CAST(2 AS TINYINT)#x] +- Project [cast(2 as tinyint) AS CAST(2 AS TINYINT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1069,13 +1077,14 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS SMALLINT)#x as float) AS CAST(2 AS SMALLINT)#x] + :- Project [cast(CAST(1 AS FLOAT)#x as double) AS CAST(1 AS FLOAT)#x] + : +- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS SMALLINT)#x as double) AS CAST(2 AS SMALLINT)#x] +- Project [cast(2 as smallint) AS CAST(2 AS SMALLINT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1089,13 +1098,14 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS INT)#x as float) AS CAST(2 AS INT)#x] + :- Project [cast(CAST(1 AS FLOAT)#x as double) AS CAST(1 AS FLOAT)#x] + : +- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS INT)#x as double) AS CAST(2 AS INT)#x] +- Project [cast(2 as int) AS CAST(2 AS INT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1109,13 +1119,14 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS BIGINT)#xL as float) AS CAST(2 AS BIGINT)#x] + :- Project [cast(CAST(1 AS FLOAT)#x as double) AS CAST(1 AS FLOAT)#x] + : +- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS BIGINT)#xL as double) AS CAST(2 AS BIGINT)#x] +- Project [cast(2 as bigint) AS CAST(2 AS BIGINT)#xL] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1189,19 +1200,20 @@ SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS FLOAT)#x as string) AS CAST(1 AS FLOAT)#x] + :- Project [cast(CAST(1 AS FLOAT)#x as double) AS CAST(1 AS FLOAT)#x] : +- Project [cast(1 as float) AS CAST(1 AS FLOAT)#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as double) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1448,19 +1460,19 @@ SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS DOUBLE)#x as string) AS CAST(1 AS DOUBLE)#x] - : +- Project [cast(1 as double) AS CAST(1 AS DOUBLE)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(1 as double) AS CAST(1 AS DOUBLE)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as double) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1709,19 +1721,20 @@ SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(1 AS DECIMAL(10,0))#x as string) AS CAST(1 AS DECIMAL(10,0))#x] + :- Project [cast(CAST(1 AS DECIMAL(10,0))#x as double) AS CAST(1 AS DECIMAL(10,0))#x] : +- Project [cast(1 as decimal(10,0)) AS CAST(1 AS DECIMAL(10,0))#x] : +- SubqueryAlias t : +- View (`t`, [1#x]) : +- Project [cast(1#x as int) AS 1#x] : +- Project [1 AS 1#x] : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as double) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1829,13 +1842,14 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS TINYINT)#x as string) AS CAST(2 AS TINYINT)#x] + :- Project [cast(CAST(1 AS STRING)#x as bigint) AS CAST(1 AS STRING)#xL] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS TINYINT)#x as bigint) AS CAST(2 AS TINYINT)#xL] +- Project [cast(2 as tinyint) AS CAST(2 AS TINYINT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1849,13 +1863,14 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS SMALLINT)#x as string) AS CAST(2 AS SMALLINT)#x] + :- Project [cast(CAST(1 AS STRING)#x as bigint) AS CAST(1 AS STRING)#xL] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS SMALLINT)#x as bigint) AS CAST(2 AS SMALLINT)#xL] +- Project [cast(2 as smallint) AS CAST(2 AS SMALLINT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1869,13 +1884,14 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS INT)#x as string) AS CAST(2 AS INT)#x] + :- Project [cast(CAST(1 AS STRING)#x as bigint) AS CAST(1 AS STRING)#xL] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS INT)#x as bigint) AS CAST(2 AS INT)#xL] +- Project [cast(2 as int) AS CAST(2 AS INT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1889,19 +1905,19 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS BIGINT)#xL as string) AS CAST(2 AS BIGINT)#x] - +- Project [cast(2 as bigint) AS CAST(2 AS BIGINT)#xL] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(CAST(1 AS STRING)#x as bigint) AS CAST(1 AS STRING)#xL] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2 as bigint) AS CAST(2 AS BIGINT)#xL] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1909,13 +1925,14 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS FLOAT)#x as string) AS CAST(2 AS FLOAT)#x] + :- Project [cast(CAST(1 AS STRING)#x as double) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS FLOAT)#x as double) AS CAST(2 AS FLOAT)#x] +- Project [cast(2 as float) AS CAST(2 AS FLOAT)#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1929,19 +1946,19 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS DOUBLE)#x as string) AS CAST(2 AS DOUBLE)#x] - +- Project [cast(2 as double) AS CAST(2 AS DOUBLE)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(CAST(1 AS STRING)#x as double) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2 as double) AS CAST(2 AS DOUBLE)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -1949,13 +1966,14 @@ SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2 AS DECIMAL(10,0))#x as string) AS CAST(2 AS DECIMAL(10,0))#x] + :- Project [cast(CAST(1 AS STRING)#x as double) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS DECIMAL(10,0))#x as double) AS CAST(2 AS DECIMAL(10,0))#x] +- Project [cast(2 as decimal(10,0)) AS CAST(2 AS DECIMAL(10,0))#x] +- SubqueryAlias t +- View (`t`, [1#x]) @@ -1986,51 +2004,41 @@ Distinct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", - "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"BINARY\"", - "dataType2" : "\"STRING\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t" - } ] -} +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS STRING)#x as binary) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2 as binary) AS CAST(2 AS BINARY)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", - "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"BOOLEAN\"", - "dataType2" : "\"STRING\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t" - } ] -} +Distinct ++- Union false, false + :- Project [cast(CAST(1 AS STRING)#x as boolean) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2 as boolean) AS CAST(2 AS BOOLEAN)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2038,19 +2046,19 @@ SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as tim -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)#x as string) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)#x] - +- Project [cast(2017-12-11 09:30:00.0 as timestamp) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(CAST(1 AS STRING)#x as timestamp) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2017-12-11 09:30:00.0 as timestamp) AS CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2058,19 +2066,19 @@ SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) -- !query analysis Distinct +- Union false, false - :- Project [cast(1 as string) AS CAST(1 AS STRING)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(CAST(2017-12-11 09:30:00 AS DATE)#x as string) AS CAST(2017-12-11 09:30:00 AS DATE)#x] - +- Project [cast(2017-12-11 09:30:00 as date) AS CAST(2017-12-11 09:30:00 AS DATE)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(CAST(1 AS STRING)#x as date) AS CAST(1 AS STRING)#x] + : +- Project [cast(1 as string) AS CAST(1 AS STRING)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(2017-12-11 09:30:00 as date) AS CAST(2017-12-11 09:30:00 AS DATE)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2251,26 +2259,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", - "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"STRING\"", - "dataType2" : "\"BINARY\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t" - } ] -} +Distinct ++- Union false, false + :- Project [cast(1 as binary) AS CAST(1 AS BINARY)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as binary) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2545,26 +2548,21 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", - "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"STRING\"", - "dataType2" : "\"BOOLEAN\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t" - } ] -} +Distinct ++- Union false, false + :- Project [cast(1 as boolean) AS CAST(1 AS BOOLEAN)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as boolean) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -2841,19 +2839,19 @@ SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP)#x as string) AS CAST(2017-12-12 09:30:00.0 AS TIMESTAMP)#x] - : +- Project [cast(2017-12-12 09:30:00.0 as timestamp) AS CAST(2017-12-12 09:30:00.0 AS TIMESTAMP)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(2017-12-12 09:30:00.0 as timestamp) AS CAST(2017-12-12 09:30:00.0 AS TIMESTAMP)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as timestamp) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query @@ -3125,19 +3123,19 @@ SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) -- !query analysis Distinct +- Union false, false - :- Project [cast(CAST(2017-12-12 09:30:00 AS DATE)#x as string) AS CAST(2017-12-12 09:30:00 AS DATE)#x] - : +- Project [cast(2017-12-12 09:30:00 as date) AS CAST(2017-12-12 09:30:00 AS DATE)#x] - : +- SubqueryAlias t - : +- View (`t`, [1#x]) - : +- Project [cast(1#x as int) AS 1#x] - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] - +- SubqueryAlias t - +- View (`t`, [1#x]) - +- Project [cast(1#x as int) AS 1#x] - +- Project [1 AS 1#x] - +- OneRowRelation + :- Project [cast(2017-12-12 09:30:00 as date) AS CAST(2017-12-12 09:30:00 AS DATE)#x] + : +- SubqueryAlias t + : +- View (`t`, [1#x]) + : +- Project [cast(1#x as int) AS 1#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [cast(CAST(2 AS STRING)#x as date) AS CAST(2 AS STRING)#x] + +- Project [cast(2 as string) AS CAST(2 AS STRING)#x] + +- SubqueryAlias t + +- View (`t`, [1#x]) + +- Project [cast(1#x as int) AS 1#x] + +- Project [1 AS 1#x] + +- OneRowRelation -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out index 170e7dff38ac3..d516b82508de5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/typeCoercion/native/windowFrameCoercion.sql.out @@ -301,13 +301,12 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BET -- !query analysis org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", "messageParameters" : { - "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "exprType" : "\"BINARY\"", - "location" : "upper", - "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + "orderSpecType" : "\"BINARY\"", + "sqlExpr" : "\"(PARTITION BY 1 ORDER BY CAST(1 AS BINARY) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-cross-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-cross-join.sql.out index c5ee1742f5d7c..391eb371d0d54 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-cross-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-cross-join.sql.out @@ -95,7 +95,7 @@ Project [k#x, v1#x, k#x, v2#x] SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" -- !query analysis Project [k#x, v1#x, k#x, v2#x] -+- Filter ((cast(udf(cast(v1#x as string)) as int) = cast(1 as int)) AND (cast(udf(cast(v2#x as string)) as int) = cast(22 as int))) ++- Filter ((cast(cast(udf(cast(v1#x as string)) as int) as bigint) = cast(1 as bigint)) AND (cast(cast(udf(cast(v2#x as string)) as int) as bigint) = cast(22 as bigint))) +- Join Cross :- SubqueryAlias nt1 : +- View (`nt1`, [k#x, v1#x]) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out index 5811a4ff6566c..e016a8e4fab1e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-group-by.sql.out @@ -619,25 +619,8 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT udf(every("true")) -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"true\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"BOOLEAN\"", - "sqlExpr" : "\"every(true)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 24, - "fragment" : "every(\"true\")" - } ] -} +Aggregate [cast(udf(cast(every(cast(true as boolean)) as string)) as boolean) AS udf(every(true))#x] ++- OneRowRelation -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-union.sql.out index a1436d0a77c83..df316707c7261 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/udf-union.sql.out @@ -40,31 +40,32 @@ Project [cast(udf(cast(c1#x as string)) as int) AS c1#x, cast(udf(cast(c2#x as s -- !query SELECT udf(c1) as c1, udf(c2) as c2 -FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a' UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2) -- !query analysis -Project [cast(udf(cast(c1#x as string)) as decimal(11,1)) AS c1#x, cast(udf(cast(c2#x as string)) as string) AS c2#x] +Project [cast(udf(cast(c1#x as string)) as decimal(11,1)) AS c1#x, cast(udf(cast(c2#xL as string)) as bigint) AS c2#xL] +- SubqueryAlias __auto_generated_subquery_name +- Union false, false :- Union false, false - : :- Project [cast(c1#x as decimal(11,1)) AS c1#x, c2#x] + : :- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : : +- Project [cast(udf(cast(c1#x as string)) as int) AS c1#x, cast(udf(cast(c2#x as string)) as string) AS c2#x] - : : +- SubqueryAlias t1 - : : +- View (`t1`, [c1#x, c2#x]) - : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as string) AS c2#x] - : : +- SubqueryAlias tbl - : : +- LocalRelation [c1#x, c2#x] - : +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as string) AS c2#x] + : : +- Filter (c2#x = a) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [c1#x, c2#x]) + : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as string) AS c2#x] + : : +- SubqueryAlias tbl + : : +- LocalRelation [c1#x, c2#x] + : +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : +- Project [cast(udf(cast(c1#x as string)) as decimal(2,1)) AS c1#x, cast(udf(cast(c2#x as string)) as int) AS c2#x] : +- SubqueryAlias t2 : +- View (`t2`, [c1#x, c2#x]) : +- Project [cast(c1#x as decimal(2,1)) AS c1#x, cast(c2#x as int) AS c2#x] : +- SubqueryAlias tbl : +- LocalRelation [c1#x, c2#x] - +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as string) AS c2#x] + +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] +- Project [cast(udf(cast(c1#x as string)) as decimal(2,1)) AS c1#x, cast(udf(cast(c2#x as string)) as int) AS c2#x] +- SubqueryAlias t2 +- View (`t2`, [c1#x, c2#x]) @@ -159,10 +160,10 @@ UNION ALL SELECT map(1, 2, 3, NULL), udf(1) -- !query analysis Union false, false -:- Project [cast(map(1, 2)#x as map) AS map(1, 2)#x, str#x] +:- Project [cast(map(1, 2)#x as map) AS map(1, 2)#x, cast(str#x as bigint) AS str#xL] : +- Project [map(1, 2) AS map(1, 2)#x, cast(udf(cast(str as string)) as string) AS str#x] : +- OneRowRelation -+- Project [map(1, 2, 3, NULL)#x, cast(udf(1)#x as string) AS udf(1)#x] ++- Project [map(1, 2, 3, NULL)#x, cast(udf(1)#x as bigint) AS udf(1)#xL] +- Project [map(1, 2, 3, cast(null as int)) AS map(1, 2, 3, NULL)#x, cast(udf(cast(1 as string)) as int) AS udf(1)#x] +- OneRowRelation @@ -173,10 +174,10 @@ UNION ALL SELECT array(1, 2, 3, NULL), udf(1) -- !query analysis Union false, false -:- Project [cast(array(1, 2)#x as array) AS array(1, 2)#x, str#x] +:- Project [cast(array(1, 2)#x as array) AS array(1, 2)#x, cast(str#x as bigint) AS str#xL] : +- Project [array(1, 2) AS array(1, 2)#x, cast(udf(cast(str as string)) as string) AS str#x] : +- OneRowRelation -+- Project [array(1, 2, 3, NULL)#x, cast(udf(1)#x as string) AS udf(1)#x] ++- Project [array(1, 2, 3, NULL)#x, cast(udf(1)#x as bigint) AS udf(1)#xL] +- Project [array(1, 2, 3, cast(null as int)) AS array(1, 2, 3, NULL)#x, cast(udf(cast(1 as string)) as int) AS udf(1)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out index cafdd850e86d6..93456003254b8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out @@ -46,25 +46,25 @@ FROM (SELECT * FROM t1 UNION ALL SELECT * FROM t2) -- !query analysis -Project [c1#x, c2#x] +Project [c1#x, c2#xL] +- SubqueryAlias __auto_generated_subquery_name +- Union false, false :- Union false, false - : :- Project [cast(c1#x as decimal(11,1)) AS c1#x, c2#x] + : :- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t1 : : +- View (`t1`, [c1#x, c2#x]) : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as string) AS c2#x] : : +- SubqueryAlias tbl : : +- LocalRelation [c1#x, c2#x] - : +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as string) AS c2#x] + : +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 : +- View (`t2`, [c1#x, c2#x]) : +- Project [cast(c1#x as decimal(2,1)) AS c1#x, cast(c2#x as int) AS c2#x] : +- SubqueryAlias tbl : +- LocalRelation [c1#x, c2#x] - +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as string) AS c2#x] + +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] +- Project [c1#x, c2#x] +- SubqueryAlias t2 +- View (`t2`, [c1#x, c2#x]) @@ -159,10 +159,10 @@ UNION ALL SELECT map(1, 2, 3, NULL), 1 -- !query analysis Union false, false -:- Project [cast(map(1, 2)#x as map) AS map(1, 2)#x, str#x] +:- Project [cast(map(1, 2)#x as map) AS map(1, 2)#x, cast(str#x as bigint) AS str#xL] : +- Project [map(1, 2) AS map(1, 2)#x, str AS str#x] : +- OneRowRelation -+- Project [map(1, 2, 3, NULL)#x, cast(1#x as string) AS 1#x] ++- Project [map(1, 2, 3, NULL)#x, cast(1#x as bigint) AS 1#xL] +- Project [map(1, 2, 3, cast(null as int)) AS map(1, 2, 3, NULL)#x, 1 AS 1#x] +- OneRowRelation @@ -173,10 +173,10 @@ UNION ALL SELECT array(1, 2, 3, NULL), 1 -- !query analysis Union false, false -:- Project [cast(array(1, 2)#x as array) AS array(1, 2)#x, str#x] +:- Project [cast(array(1, 2)#x as array) AS array(1, 2)#x, cast(str#x as bigint) AS str#xL] : +- Project [array(1, 2) AS array(1, 2)#x, str AS str#x] : +- OneRowRelation -+- Project [array(1, 2, 3, NULL)#x, cast(1#x as string) AS 1#x] ++- Project [array(1, 2, 3, NULL)#x, cast(1#x as bigint) AS 1#xL] +- Project [array(1, 2, 3, cast(null as int)) AS array(1, 2, 3, NULL)#x, 1 AS 1#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/url-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/url-functions.sql.out index 9f213bcb8c914..bdf90f6a0ed14 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/url-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/url-functions.sql.out @@ -2,56 +2,56 @@ -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, HOST, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, HOST)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, HOST, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, HOST)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PATH, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PATH)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PATH, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PATH)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, QUERY, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, QUERY)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, QUERY, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, QUERY)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, REF, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, REF)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, REF, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, REF)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PROTOCOL, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PROTOCOL)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PROTOCOL, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, PROTOCOL)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, FILE, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, FILE)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, FILE, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, FILE)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, AUTHORITY, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, AUTHORITY)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, AUTHORITY, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, AUTHORITY)#x] +- OneRowRelation -- !query select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO') -- !query analysis -Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, USERINFO, false) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, USERINFO)#x] +Project [parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, USERINFO, true) AS parse_url(http://userinfo@spark.apache.org/path?query=1#Ref, USERINFO)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out index 12a90ce32cc68..367d5b0167014 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -268,11 +268,11 @@ SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY t RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ORDER BY cate, to_timestamp_ntz(val_timestamp) -- !query analysis -Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) ASC NULLS FIRST], true +Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) ASC NULLS FIRST], true +- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x] +- Project [val_timestamp#x, cate#x, _w0#x, _w1#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x] +- Window [avg(_w0#x) windowspecdefinition(cate#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '23 04' DAY TO HOUR)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '23 04' DAY TO HOUR FOLLOWING)#x], [cate#x], [_w1#x ASC NULLS FIRST] - +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) AS _w1#x] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) AS _w1#x] +- SubqueryAlias testdata +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) +- Project [cast(val#x as int) AS val#x, cast(val_long#xL as bigint) AS val_long#xL, cast(val_double#x as double) AS val_double#x, cast(val_date#x as date) AS val_date#x, cast(val_timestamp#x as timestamp) AS val_timestamp#x, cast(cate#x as string) AS cate#x] @@ -304,11 +304,11 @@ SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY t RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData ORDER BY cate, to_timestamp_ntz(val_timestamp) -- !query analysis -Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) ASC NULLS FIRST], true +Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) ASC NULLS FIRST], true +- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x] +- Project [val_timestamp#x, cate#x, _w0#x, _w1#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x] +- Window [avg(_w0#x) windowspecdefinition(cate#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '1-1' YEAR TO MONTH)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1-1' YEAR TO MONTH FOLLOWING)#x], [cate#x], [_w1#x ASC NULLS FIRST] - +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) AS _w1#x] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) AS _w1#x] +- SubqueryAlias testdata +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) +- Project [cast(val#x as int) AS val#x, cast(val_long#xL as bigint) AS val_long#xL, cast(val_double#x as double) AS val_double#x, cast(val_date#x as date) AS val_date#x, cast(val_timestamp#x as timestamp) AS val_timestamp#x, cast(cate#x as string) AS cate#x] @@ -340,11 +340,11 @@ SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY t RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData ORDER BY cate, to_timestamp_ntz(val_timestamp) -- !query analysis -Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) ASC NULLS FIRST], true +Sort [cate#x ASC NULLS FIRST, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) ASC NULLS FIRST], true +- Project [val_timestamp#x, cate#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)#x] +- Project [val_timestamp#x, cate#x, _w0#x, _w1#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)#x, avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)#x] +- Window [avg(_w0#x) windowspecdefinition(cate#x, _w1#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, currentrow$(), INTERVAL '1 02:03:04.001' DAY TO SECOND)) AS avg(val_timestamp) OVER (PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)#x], [cate#x], [_w1#x ASC NULLS FIRST] - +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), false) AS _w1#x] + +- Project [val_timestamp#x, cate#x, cast(val_timestamp#x as double) AS _w0#x, to_timestamp_ntz(val_timestamp#x, None, TimestampNTZType, Some(America/Los_Angeles), true) AS _w1#x] +- SubqueryAlias testdata +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) +- Project [cast(val#x as int) AS val#x, cast(val_long#xL as bigint) AS val_long#xL, cast(val_double#x as double) AS val_double#x, cast(val_date#x as date) AS val_date#x, cast(val_timestamp#x as timestamp) AS val_timestamp#x, cast(cate#x as string) AS cate#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/xml-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/xml-functions.sql.out index de9fb2f395210..e0a249e4cf3fe 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/xml-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/xml-functions.sql.out @@ -9,7 +9,7 @@ Project [to_xml((indent,), named_struct(a, 1, b, 2), Some(America/Los_Angeles)) -- !query select to_xml(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy', 'indent', '')) -- !query analysis -Project [to_xml((timestampFormat,dd/MM/yyyy), (indent,), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), false)), Some(America/Los_Angeles)) AS to_xml(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +Project [to_xml((timestampFormat,dd/MM/yyyy), (indent,), named_struct(time, to_timestamp(2015-08-26, Some(yyyy-MM-dd), TimestampType, Some(America/Los_Angeles), true)), Some(America/Los_Angeles)) AS to_xml(named_struct(time, to_timestamp(2015-08-26, yyyy-MM-dd)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index 0775b9780332c..13bbd9d81b799 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -88,7 +88,7 @@ SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)); set spark.sql.decimalOperations.allowPrecisionLoss=false; -- test decimal operations -select id, a+b, a-b, a*b, a/b from decimals_test order by id; +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id; -- test operations between decimals and constants select id, a*10, b/10 from decimals_test order by id; diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/array.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/array.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/cast.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/cast.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/cast.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/date.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/date.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/date.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/date.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/datetime-parsing-invalid.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/datetime-parsing-invalid.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/datetime-parsing-invalid.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/datetime-parsing-invalid.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/datetime-special.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/datetime-special.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/datetime-special.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/datetime-special.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-enabled.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-enabled.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/higher-order-functions.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/higher-order-functions.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/higher-order-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/interval.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/interval.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/keywords.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/keywords.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/literals.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/literals.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/literals.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/map.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/map.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/math.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/math.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/parse-schema-string.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/parse-schema-string.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/parse-schema-string.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/parse-schema-string.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/string-functions.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/string-functions.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/string-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/timestamp.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/timestamp.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/timestamp.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/try_aggregates.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/try_aggregates.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/try_aggregates.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/try_aggregates.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/try_arithmetic.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/try_arithmetic.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/try_arithmetic.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/try_arithmetic.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/try_datetime_functions.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/try_datetime_functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/try_element_at.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/try_element_at.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/ansi/try_element_at.sql rename to sql/core/src/test/resources/sql-tests/inputs/nonansi/try_element_at.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index 8bca7144c0a98..6261bc93b1856 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -548,7 +548,8 @@ values (0, 'abc') tab(x, y) -- Union distinct with a VALUES list. values (0, 1) tab(x, y) -|> union table t; +|> union table t +|> where x = 0; -- Union all with a table subquery on both the source and target sides. (select * from t) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-union.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-union.sql index 883369705c3cb..207bf557acb0c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-union.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-union.sql @@ -11,7 +11,7 @@ FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 -- Type Coerced Union SELECT udf(c1) as c1, udf(c2) as c2 -FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a' UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 UNION ALL diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out deleted file mode 100644 index 0dbdf1d9975c9..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out +++ /dev/null @@ -1,1999 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -SELECT CAST('1.23' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1.23'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 26, - "fragment" : "CAST('1.23' AS int)" - } ] -} - - --- !query -SELECT CAST('1.23' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1.23'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "CAST('1.23' AS long)" - } ] -} - - --- !query -SELECT CAST('-4.56' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'-4.56'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "CAST('-4.56' AS int)" - } ] -} - - --- !query -SELECT CAST('-4.56' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'-4.56'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "CAST('-4.56' AS long)" - } ] -} - - --- !query -SELECT CAST('abc' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "CAST('abc' AS int)" - } ] -} - - --- !query -SELECT CAST('abc' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 26, - "fragment" : "CAST('abc' AS long)" - } ] -} - - --- !query -SELECT CAST('abc' AS float) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "CAST('abc' AS float)" - } ] -} - - --- !query -SELECT CAST('abc' AS double) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "CAST('abc' AS double)" - } ] -} - - --- !query -SELECT CAST('1234567890123' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1234567890123'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "CAST('1234567890123' AS int)" - } ] -} - - --- !query -SELECT CAST('12345678901234567890123' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'12345678901234567890123'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 46, - "fragment" : "CAST('12345678901234567890123' AS long)" - } ] -} - - --- !query -SELECT CAST('' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "''", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 22, - "fragment" : "CAST('' AS int)" - } ] -} - - --- !query -SELECT CAST('' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "''", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "CAST('' AS long)" - } ] -} - - --- !query -SELECT CAST('' AS float) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "''", - "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "CAST('' AS float)" - } ] -} - - --- !query -SELECT CAST('' AS double) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "''", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "CAST('' AS double)" - } ] -} - - --- !query -SELECT CAST(NULL AS int) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT CAST(NULL AS long) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT CAST('123.a' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'123.a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "CAST('123.a' AS int)" - } ] -} - - --- !query -SELECT CAST('123.a' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'123.a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "CAST('123.a' AS long)" - } ] -} - - --- !query -SELECT CAST('123.a' AS float) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'123.a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"FLOAT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 29, - "fragment" : "CAST('123.a' AS float)" - } ] -} - - --- !query -SELECT CAST('123.a' AS double) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'123.a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "CAST('123.a' AS double)" - } ] -} - - --- !query -SELECT CAST('-2147483648' AS int) --- !query schema -struct --- !query output --2147483648 - - --- !query -SELECT CAST('-2147483649' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'-2147483649'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 33, - "fragment" : "CAST('-2147483649' AS int)" - } ] -} - - --- !query -SELECT CAST('2147483647' AS int) --- !query schema -struct --- !query output -2147483647 - - --- !query -SELECT CAST('2147483648' AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'2147483648'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 32, - "fragment" : "CAST('2147483648' AS int)" - } ] -} - - --- !query -SELECT CAST('-9223372036854775808' AS long) --- !query schema -struct --- !query output --9223372036854775808 - - --- !query -SELECT CAST('-9223372036854775809' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'-9223372036854775809'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "CAST('-9223372036854775809' AS long)" - } ] -} - - --- !query -SELECT CAST('9223372036854775807' AS long) --- !query schema -struct --- !query output -9223372036854775807 - - --- !query -SELECT CAST('9223372036854775808' AS long) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'9223372036854775808'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 42, - "fragment" : "CAST('9223372036854775808' AS long)" - } ] -} - - --- !query -SELECT HEX(CAST('abc' AS binary)) --- !query schema -struct --- !query output -616263 - - --- !query -SELECT HEX(CAST(CAST(123 AS byte) AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 44, - "fragment" : "CAST(CAST(123 AS byte) AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 45, - "fragment" : "CAST(CAST(-123 AS byte) AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(123S AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"SMALLINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(123S AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(-123S AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"SMALLINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 32, - "fragment" : "CAST(-123S AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(123 AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"INT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 30, - "fragment" : "CAST(123 AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(-123 AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"INT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(-123 AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(123L AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(123 AS BINARY)\"", - "srcType" : "\"BIGINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 31, - "fragment" : "CAST(123L AS binary)" - } ] -} - - --- !query -SELECT HEX(CAST(-123L AS binary)) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(-123 AS BINARY)\"", - "srcType" : "\"BIGINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 32, - "fragment" : "CAST(-123L AS binary)" - } ] -} - - --- !query -DESC FUNCTION boolean --- !query schema -struct --- !query output -Class: org.apache.spark.sql.catalyst.expressions.Cast -Function: boolean -Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. - - --- !query -DESC FUNCTION EXTENDED boolean --- !query schema -struct --- !query output -Class: org.apache.spark.sql.catalyst.expressions.Cast -Extended Usage: - No example/argument for boolean. - - Since: 2.0.1 - -Function: boolean -Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. - - --- !query -SELECT CAST('interval 3 month 1 hour' AS interval) --- !query schema -struct --- !query output -3 months 1 hours - - --- !query -SELECT CAST("interval '3-1' year to month" AS interval year to month) --- !query schema -struct --- !query output -3-1 - - --- !query -SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) --- !query schema -struct --- !query output -3 00:00:01.000000000 - - --- !query -SELECT CAST(interval 3 month 1 hour AS string) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "_LEGACY_ERROR_TEMP_0029", - "messageParameters" : { - "literal" : "interval 3 month 1 hour" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 35, - "fragment" : "interval 3 month 1 hour" - } ] -} - - --- !query -SELECT CAST(interval 3 year 1 month AS string) --- !query schema -struct --- !query output -INTERVAL '3-1' YEAR TO MONTH - - --- !query -SELECT CAST(interval 3 day 1 second AS string) --- !query schema -struct --- !query output -INTERVAL '3 00:00:01' DAY TO SECOND - - --- !query -select cast(' 1' as tinyint) --- !query schema -struct --- !query output -1 - - --- !query -select cast(' 1\t' as tinyint) --- !query schema -struct --- !query output -1 - - --- !query -select cast(' 1' as smallint) --- !query schema -struct --- !query output -1 - - --- !query -select cast(' 1' as INT) --- !query schema -struct --- !query output -1 - - --- !query -select cast(' 1' as bigint) --- !query schema -struct --- !query output -1 - - --- !query -select cast(' 1' as float) --- !query schema -struct --- !query output -1.0 - - --- !query -select cast(' 1 ' as DOUBLE) --- !query schema -struct --- !query output -1.0 - - --- !query -select cast('1.0 ' as DEC) --- !query schema -struct --- !query output -1 - - --- !query -select cast('1中文' as tinyint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1中文'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TINYINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 29, - "fragment" : "cast('1中文' as tinyint)" - } ] -} - - --- !query -select cast('1中文' as smallint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1中文'", - "sourceType" : "\"STRING\"", - "targetType" : "\"SMALLINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "cast('1中文' as smallint)" - } ] -} - - --- !query -select cast('1中文' as INT) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1中文'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "cast('1中文' as INT)" - } ] -} - - --- !query -select cast('中文1' as bigint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'中文1'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "cast('中文1' as bigint)" - } ] -} - - --- !query -select cast('1中文' as bigint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1中文'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "cast('1中文' as bigint)" - } ] -} - - --- !query -select cast('\t\t true \n\r ' as boolean) --- !query schema -struct --- !query output -true - - --- !query -select cast('\t\n false \t\r' as boolean) --- !query schema -struct --- !query output -false - - --- !query -select cast('\t\n xyz \t\r' as boolean) --- !query schema -struct<> --- !query output -org.apache.spark.SparkRuntimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'\t\n xyz \t\r'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BOOLEAN\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "cast('\\t\\n xyz \\t\\r' as boolean)" - } ] -} - - --- !query -select cast('23.45' as decimal(4, 2)) --- !query schema -struct --- !query output -23.45 - - --- !query -select cast('123.45' as decimal(4, 2)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "4", - "scale" : "2", - "value" : "123.45" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "cast('123.45' as decimal(4, 2))" - } ] -} - - --- !query -select cast('xyz' as decimal(4, 2)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'xyz'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DECIMAL(4,2)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "cast('xyz' as decimal(4, 2))" - } ] -} - - --- !query -select cast('2022-01-01' as date) --- !query schema -struct --- !query output -2022-01-01 - - --- !query -select cast('a' as date) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "cast('a' as date)" - } ] -} - - --- !query -select cast('2022-01-01 00:00:00' as timestamp) --- !query schema -struct --- !query output -2022-01-01 00:00:00 - - --- !query -select cast('a' as timestamp) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 29, - "fragment" : "cast('a' as timestamp)" - } ] -} - - --- !query -select cast('2022-01-01 00:00:00' as timestamp_ntz) --- !query schema -struct --- !query output -2022-01-01 00:00:00 - - --- !query -select cast('a' as timestamp_ntz) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP_NTZ\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 33, - "fragment" : "cast('a' as timestamp_ntz)" - } ] -} - - --- !query -select cast(cast('inf' as double) as timestamp) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "Infinity", - "sourceType" : "\"DOUBLE\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 47, - "fragment" : "cast(cast('inf' as double) as timestamp)" - } ] -} - - --- !query -select cast(cast('inf' as float) as timestamp) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "Infinity", - "sourceType" : "\"DOUBLE\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 46, - "fragment" : "cast(cast('inf' as float) as timestamp)" - } ] -} - - --- !query -select cast(interval '1' year as tinyint) --- !query schema -struct --- !query output -1 - - --- !query -select cast(interval '-10-2' year to month as smallint) --- !query schema -struct --- !query output --122 - - --- !query -select cast(interval '1000' month as int) --- !query schema -struct --- !query output -1000 - - --- !query -select cast(interval -'10.123456' second as tinyint) --- !query schema -struct --- !query output --10 - - --- !query -select cast(interval '23:59:59' hour to second as smallint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"INTERVAL HOUR TO SECOND\"", - "targetType" : "\"SMALLINT\"", - "value" : "INTERVAL '23:59:59' HOUR TO SECOND" - } -} - - --- !query -select cast(interval -'1 02:03:04.123' day to second as int) --- !query schema -struct --- !query output --93784 - - --- !query -select cast(interval '10' day as bigint) --- !query schema -struct --- !query output -10 - - --- !query -select cast(interval '-1000' month as tinyint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"INTERVAL MONTH\"", - "targetType" : "\"TINYINT\"", - "value" : "INTERVAL '-1000' MONTH" - } -} - - --- !query -select cast(interval '1000000' second as smallint) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"INTERVAL SECOND\"", - "targetType" : "\"SMALLINT\"", - "value" : "INTERVAL '1000000' SECOND" - } -} - - --- !query -select cast(1Y as interval year) --- !query schema -struct --- !query output -1-0 - - --- !query -select cast(-122S as interval year to month) --- !query schema -struct --- !query output --10-2 - - --- !query -select cast(ym as interval year to month) from values(-122S) as t(ym) --- !query schema -struct --- !query output --10-2 - - --- !query -select cast(1000 as interval month) --- !query schema -struct --- !query output -83-4 - - --- !query -select cast(-10L as interval second) --- !query schema -struct --- !query output --0 00:00:10.000000000 - - --- !query -select cast(100Y as interval hour to second) --- !query schema -struct --- !query output -0 00:01:40.000000000 - - --- !query -select cast(dt as interval hour to second) from values(100Y) as t(dt) --- !query schema -struct --- !query output -0 00:01:40.000000000 - - --- !query -select cast(-1000S as interval day to second) --- !query schema -struct --- !query output --0 00:16:40.000000000 - - --- !query -select cast(10 as interval day) --- !query schema -struct --- !query output -10 00:00:00.000000000 - - --- !query -select cast(2147483647 as interval year) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"INT\"", - "targetType" : "\"INTERVAL YEAR\"", - "value" : "2147483647" - } -} - - --- !query -select cast(-9223372036854775808L as interval day) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"BIGINT\"", - "targetType" : "\"INTERVAL DAY\"", - "value" : "-9223372036854775808L" - } -} - - --- !query -select cast(interval '-1' year as decimal(10, 0)) --- !query schema -struct --- !query output --1 - - --- !query -select cast(interval '1.000001' second as decimal(10, 6)) --- !query schema -struct --- !query output -1.000001 - - --- !query -select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) --- !query schema -struct --- !query output -29470.0010 - - --- !query -select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) --- !query schema -struct --- !query output -90123.1 - - --- !query -select cast(interval '10.123' second as decimal(4, 2)) --- !query schema -struct --- !query output -10.12 - - --- !query -select cast(interval '10.005' second as decimal(4, 2)) --- !query schema -struct --- !query output -10.01 - - --- !query -select cast(interval '10.123' second as decimal(5, 2)) --- !query schema -struct --- !query output -10.12 - - --- !query -select cast(interval '10.123' second as decimal(1, 0)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "1", - "scale" : "0", - "value" : "10.123000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "cast(interval '10.123' second as decimal(1, 0))" - } ] -} - - --- !query -select cast(10.123456BD as interval day to second) --- !query schema -struct --- !query output -0 00:00:10.123456000 - - --- !query -select cast(80.654321BD as interval hour to minute) --- !query schema -struct --- !query output -0 01:20:00.000000000 - - --- !query -select cast(-10.123456BD as interval year to month) --- !query schema -struct --- !query output --0-10 - - --- !query -select cast(10.654321BD as interval month) --- !query schema -struct --- !query output -0-11 - - --- !query -SELECT '1.23' :: int --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1.23'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "'1.23' :: int" - } ] -} - - --- !query -SELECT 'abc' :: int --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 19, - "fragment" : "'abc' :: int" - } ] -} - - --- !query -SELECT '12345678901234567890123' :: long --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'12345678901234567890123'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 40, - "fragment" : "'12345678901234567890123' :: long" - } ] -} - - --- !query -SELECT '' :: int --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "''", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 16, - "fragment" : "'' :: int" - } ] -} - - --- !query -SELECT NULL :: int --- !query schema -struct --- !query output -NULL - - --- !query -SELECT '123.a' :: int --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'123.a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 21, - "fragment" : "'123.a' :: int" - } ] -} - - --- !query -SELECT '-2147483648' :: int --- !query schema -struct --- !query output --2147483648 - - --- !query -SELECT HEX('abc' :: binary) --- !query schema -struct --- !query output -616263 - - --- !query -SELECT HEX((123 :: byte) :: binary) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", - "sqlState" : "42K09", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "configVal" : "'false'", - "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", - "srcType" : "\"TINYINT\"", - "targetType" : "\"BINARY\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 34, - "fragment" : "(123 :: byte) :: binary" - } ] -} - - --- !query -SELECT 'interval 3 month 1 hour' :: interval --- !query schema -struct --- !query output -3 months 1 hours - - --- !query -SELECT interval 3 day 1 second :: string --- !query schema -struct --- !query output -INTERVAL '3 00:00:01' DAY TO SECOND - - --- !query -select ' 1 ' :: DOUBLE --- !query schema -struct --- !query output -1.0 - - --- !query -select '1.0 ' :: DEC --- !query schema -struct --- !query output -1 - - --- !query -select '\t\t true \n\r ' :: boolean --- !query schema -struct --- !query output -true - - --- !query -select '2022-01-01 00:00:00' :: timestamp --- !query schema -struct --- !query output -2022-01-01 00:00:00 - - --- !query -select interval '-10-2' year to month :: smallint --- !query schema -struct --- !query output --122 - - --- !query -select -10L :: interval second --- !query schema -struct --- !query output --0 00:00:10.000000000 - - --- !query -select interval '08:11:10.001' hour to second :: decimal(10, 4) --- !query schema -struct --- !query output -29470.0010 - - --- !query -select 10.123456BD :: interval day to second --- !query schema -struct --- !query output -0 00:00:10.123456000 - - --- !query -SELECT '1.23' :: int :: long --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1.23'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "'1.23' :: int" - } ] -} - - --- !query -SELECT '2147483648' :: long :: int --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"BIGINT\"", - "targetType" : "\"INT\"", - "value" : "2147483648L" - } -} - - --- !query -SELECT CAST('2147483648' :: long AS int) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "CAST_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "sourceType" : "\"BIGINT\"", - "targetType" : "\"INT\"", - "value" : "2147483648L" - } -} - - --- !query -SELECT map(1, '123', 2, '456')[1] :: int --- !query schema -struct --- !query output -123 - - --- !query -SELECT '2147483648' :: BINT --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "UNSUPPORTED_DATATYPE", - "sqlState" : "0A000", - "messageParameters" : { - "typeName" : "\"BINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 27, - "fragment" : "BINT" - } ] -} - - --- !query -SELECT '2147483648' :: SELECT --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "UNSUPPORTED_DATATYPE", - "sqlState" : "0A000", - "messageParameters" : { - "typeName" : "\"SELECT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 29, - "fragment" : "SELECT" - } ] -} - - --- !query -SELECT FALSE IS NOT NULL :: string --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'::'", - "hint" : "" - } -} diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out deleted file mode 100644 index 0708a523900ff..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out +++ /dev/null @@ -1,465 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -select to_timestamp('294248', 'y') --- !query schema -struct<> --- !query output -java.lang.ArithmeticException -long overflow - - --- !query -select to_timestamp('1', 'yy') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '1' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('-12', 'yy') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '-12' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('123', 'yy') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '123' could not be parsed, unparsed text found at index 2" - } -} - - --- !query -select to_timestamp('1', 'yyy') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '1' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('1234567', 'yyyyyyy') --- !query schema -struct<> --- !query output -org.apache.spark.SparkUpgradeException -{ - "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", - "sqlState" : "42K0B", - "messageParameters" : { - "config" : "\"spark.sql.legacy.timeParserPolicy\"", - "docroot" : "https://spark.apache.org/docs/latest", - "pattern" : "'yyyyyyy'" - } -} - - --- !query -select to_timestamp('366', 'D') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" - } -} - - --- !query -select to_timestamp('9', 'DD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '9' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('366', 'DD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" - } -} - - --- !query -select to_timestamp('9', 'DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '9' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('99', 'DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '99' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp('30-365', 'dd-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." - } -} - - --- !query -select to_timestamp('11-365', 'MM-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31." - } -} - - --- !query -select to_timestamp('2019-366', 'yyyy-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" - } -} - - --- !query -select to_timestamp('12-30-365', 'MM-dd-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." - } -} - - --- !query -select to_timestamp('2020-01-365', 'yyyy-dd-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" - } -} - - --- !query -select to_timestamp('2020-10-350', 'yyyy-MM-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" - } -} - - --- !query -select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" - } -} - - --- !query -select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) --- !query schema -struct> --- !query output -{"date":null} - - --- !query -select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" - } -} - - --- !query -select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text 'Unparseable' could not be parsed at index 0" - } -} - - --- !query -select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" - } -} - - --- !query -select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text 'Unparseable' could not be parsed at index 0" - } -} - - --- !query -select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" - } -} - - --- !query -select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text 'Unparseable' could not be parsed at index 0" - } -} - - --- !query -select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" - } -} - - --- !query -select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text 'Unparseable' could not be parsed at index 0" - } -} - - --- !query -select cast("Unparseable" as timestamp) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'Unparseable'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "cast(\"Unparseable\" as timestamp)" - } ] -} - - --- !query -select cast("Unparseable" as date) --- !query schema -struct<> --- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'Unparseable'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 34, - "fragment" : "cast(\"Unparseable\" as date)" - } ] -} diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out deleted file mode 100644 index fd30ecf203028..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out +++ /dev/null @@ -1,365 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet --- !query schema -struct<> --- !query output - - - --- !query -insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), - (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) --- !query schema -struct<> --- !query output - - - --- !query -select id, a*10, b/10 from decimals_test order by id --- !query schema -struct --- !query output -1 1000.000000000000000 99.900000000000000000 -2 123451.230000000000000 1234.512300000000000000 -3 1.234567891011000 123.410000000000000000 -4 1234567891234567890.000000000000000 0.112345678912345679 - - --- !query -select 10.3 * 3.0 --- !query schema -struct<(10.3 * 3.0):decimal(6,2)> --- !query output -30.90 - - --- !query -select 10.3000 * 3.0 --- !query schema -struct<(10.3000 * 3.0):decimal(9,5)> --- !query output -30.90000 - - --- !query -select 10.30000 * 30.0 --- !query schema -struct<(10.30000 * 30.0):decimal(11,6)> --- !query output -309.000000 - - --- !query -select 10.300000000000000000 * 3.000000000000000000 --- !query schema -struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,34)> --- !query output -30.9000000000000000000000000000000000 - - --- !query -select 10.300000000000000000 * 3.0000000000000000000 --- !query schema -struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,34)> --- !query output -30.9000000000000000000000000000000000 - - --- !query -select (5e36BD + 0.1) + 5e36BD --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "1", - "value" : "10000000000000000000000000000000000000.1" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "(5e36BD + 0.1) + 5e36BD" - } ] -} - - --- !query -select (-4e36BD - 0.1) - 7e36BD --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "1", - "value" : "-11000000000000000000000000000000000000.1" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 31, - "fragment" : "(-4e36BD - 0.1) - 7e36BD" - } ] -} - - --- !query -select 12345678901234567890.0 * 12345678901234567890.0 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "2", - "value" : "152415787532388367501905199875019052100" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "12345678901234567890.0 * 12345678901234567890.0" - } ] -} - - --- !query -select 1e35BD / 0.1 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 19, - "fragment" : "1e35BD / 0.1" - } ] -} - - --- !query -select 123456789123456789.1234567890 * 1.123456789123456789 --- !query schema -struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> --- !query output -138698367904130467.654320988515622621 - - --- !query -select 123456789123456789.1234567890 * 1.123456789123456789 --- !query schema -struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> --- !query output -138698367904130467.654320988515622621 - - --- !query -select 12345678912345.123456789123 / 0.000000012345678 --- !query schema -struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)> --- !query output -1000000073899961059796.725866332 - - --- !query -select 1.0123456789012345678901234567890123456e36BD / 0.1 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "10123456789012345678901234567890123456.000000000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e36BD / 0.1" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e35BD / 1.0 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "101234567890123456789012345678901234.560000000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e35BD / 1.0" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e34BD / 1.0 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "10123456789012345678901234567890123.456000000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e34BD / 1.0" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e33BD / 1.0 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "1012345678901234567890123456789012.345600000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e33BD / 1.0" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e32BD / 1.0 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e32BD / 1.0" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e31BD / 1.0 --- !query schema -struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)> --- !query output -10123456789012345678901234567890.123456 - - --- !query -select 1.0123456789012345678901234567890123456e31BD / 0.1 --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "38", - "scale" : "6", - "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 57, - "fragment" : "1.0123456789012345678901234567890123456e31BD / 0.1" - } ] -} - - --- !query -select 1.0123456789012345678901234567890123456e31BD / 10.0 --- !query schema -struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,6)> --- !query output -1012345678901234567890123456789.012346 - - --- !query -drop table decimals_test --- !query schema -struct<> --- !query output - diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out deleted file mode 100644 index fb60a920040e6..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out +++ /dev/null @@ -1,903 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -SELECT round(25y, 1) --- !query schema -struct --- !query output -25 - - --- !query -SELECT round(25y, 0) --- !query schema -struct --- !query output -25 - - --- !query -SELECT round(25y, -1) --- !query schema -struct --- !query output -30 - - --- !query -SELECT round(25y, -2) --- !query schema -struct --- !query output -0 - - --- !query -SELECT round(25y, -3) --- !query schema -struct --- !query output -0 - - --- !query -SELECT round(127y, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 22, - "fragment" : "round(127y, -1)" - } ] -} - - --- !query -SELECT round(-128y, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "round(-128y, -1)" - } ] -} - - --- !query -SELECT round(525s, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525s, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525s, -1) --- !query schema -struct --- !query output -530 - - --- !query -SELECT round(525s, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT round(525s, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT round(32767s, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "round(32767s, -1)" - } ] -} - - --- !query -SELECT round(-32768s, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "round(-32768s, -1)" - } ] -} - - --- !query -SELECT round(525, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525, -1) --- !query schema -struct --- !query output -530 - - --- !query -SELECT round(525, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT round(525, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT round(2147483647, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "round(2147483647, -1)" - } ] -} - - --- !query -SELECT round(-2147483647, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 29, - "fragment" : "round(-2147483647, -1)" - } ] -} - - --- !query -SELECT round(525L, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525L, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT round(525L, -1) --- !query schema -struct --- !query output -530 - - --- !query -SELECT round(525L, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT round(525L, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT round(9223372036854775807L, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "round(9223372036854775807L, -1)" - } ] -} - - --- !query -SELECT round(-9223372036854775808L, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "round(-9223372036854775808L, -1)" - } ] -} - - --- !query -SELECT bround(25y, 1) --- !query schema -struct --- !query output -25 - - --- !query -SELECT bround(25y, 0) --- !query schema -struct --- !query output -25 - - --- !query -SELECT bround(25y, -1) --- !query schema -struct --- !query output -20 - - --- !query -SELECT bround(25y, -2) --- !query schema -struct --- !query output -0 - - --- !query -SELECT bround(25y, -3) --- !query schema -struct --- !query output -0 - - --- !query -SELECT bround(127y, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 23, - "fragment" : "bround(127y, -1)" - } ] -} - - --- !query -SELECT bround(-128y, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "bround(-128y, -1)" - } ] -} - - --- !query -SELECT bround(525s, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525s, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525s, -1) --- !query schema -struct --- !query output -520 - - --- !query -SELECT bround(525s, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT bround(525s, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT bround(32767s, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "bround(32767s, -1)" - } ] -} - - --- !query -SELECT bround(-32768s, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 26, - "fragment" : "bround(-32768s, -1)" - } ] -} - - --- !query -SELECT bround(525, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525, -1) --- !query schema -struct --- !query output -520 - - --- !query -SELECT bround(525, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT bround(525, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT bround(2147483647, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 29, - "fragment" : "bround(2147483647, -1)" - } ] -} - - --- !query -SELECT bround(-2147483647, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "bround(-2147483647, -1)" - } ] -} - - --- !query -SELECT bround(525L, 1) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525L, 0) --- !query schema -struct --- !query output -525 - - --- !query -SELECT bround(525L, -1) --- !query schema -struct --- !query output -520 - - --- !query -SELECT bround(525L, -2) --- !query schema -struct --- !query output -500 - - --- !query -SELECT bround(525L, -3) --- !query schema -struct --- !query output -1000 - - --- !query -SELECT bround(9223372036854775807L, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "bround(9223372036854775807L, -1)" - } ] -} - - --- !query -SELECT bround(-9223372036854775808L, -1) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 40, - "fragment" : "bround(-9223372036854775808L, -1)" - } ] -} - - --- !query -SELECT conv('100', 2, 10) --- !query schema -struct --- !query output -4 - - --- !query -SELECT conv(-10, 16, -10) --- !query schema -struct --- !query output --16 - - --- !query -SELECT conv('9223372036854775808', 10, 16) --- !query schema -struct --- !query output -8000000000000000 - - --- !query -SELECT conv('92233720368547758070', 10, 16) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow in function conv()" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "conv('92233720368547758070', 10, 16)" - } ] -} - - --- !query -SELECT conv('9223372036854775807', 36, 10) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow in function conv()" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 42, - "fragment" : "conv('9223372036854775807', 36, 10)" - } ] -} - - --- !query -SELECT conv('-9223372036854775807', 36, 10) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : "", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "Overflow in function conv()" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "conv('-9223372036854775807', 36, 10)" - } ] -} - - --- !query -SELECT BIN(0) --- !query schema -struct --- !query output -0 - - --- !query -SELECT BIN(25) --- !query schema -struct --- !query output -11001 - - --- !query -SELECT BIN(25L) --- !query schema -struct --- !query output -11001 - - --- !query -SELECT BIN(25.5) --- !query schema -struct --- !query output -11001 - - --- !query -SELECT POSITIVE(0Y) --- !query schema -struct<(+ 0):tinyint> --- !query output -0 - - --- !query -SELECT POSITIVE(25) --- !query schema -struct<(+ 25):int> --- !query output -25 - - --- !query -SELECT POSITIVE(-25L) --- !query schema -struct<(+ -25):bigint> --- !query output --25 - - --- !query -SELECT POSITIVE(25.5) --- !query schema -struct<(+ 25.5):decimal(3,1)> --- !query output -25.5 - - --- !query -SELECT POSITIVE("25.5") --- !query schema -struct<(+ 25.5):double> --- !query output -25.5 - - --- !query -SELECT POSITIVE("invalid") --- !query schema -struct<> --- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'invalid'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 26, - "fragment" : "POSITIVE(\"invalid\")" - } ] -} - - --- !query -SELECT POSITIVE(null) --- !query schema -struct<(+ NULL):double> --- !query output -NULL diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out.java21 deleted file mode 100644 index 9d3c97baecabd..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out.java21 +++ /dev/null @@ -1,415 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) --- !query schema -struct --- !query output -30 - - --- !query -SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) --- !query schema -struct --- !query output -30.0 - - --- !query -SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) --- !query schema -struct --- !query output -25 - - --- !query -SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) --- !query schema -struct --- !query output -0-2 - - --- !query -SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) --- !query schema -struct --- !query output -0 00:00:02.000000000 - - --- !query -SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 23, - "fragment" : "col + 1L" - } ] -} - - --- !query -SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", - "sqlState" : "22012", - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", - "sqlState" : "22012", - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) --- !query schema -struct --- !query output -10.0 - - --- !query -SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) --- !query schema -struct --- !query output -10.00000 - - --- !query -SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) --- !query schema -struct --- !query output -12.5 - - --- !query -SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) --- !query schema -struct --- !query output -4.611686018427388E18 - - --- !query -SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) --- !query schema -struct --- !query output -0-1 - - --- !query -SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) --- !query schema -struct --- !query output -0 00:00:01.000000000 - - --- !query -SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 23, - "fragment" : "col + 1L" - } ] -} - - --- !query -SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", - "sqlState" : "22012", - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} - - --- !query -SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", - "sqlState" : "22012", - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index c1330c620acfb..7394e428091c7 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -178,17 +178,49 @@ struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "3", + "indexValue" : "5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "element_at(array(1, 2, 3), 5)" + } ] +} -- !query select element_at(array(1, 2, 3), -5) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "3", + "indexValue" : "-5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "element_at(array(1, 2, 3), -5)" + } ] +} -- !query @@ -199,32 +231,87 @@ struct<> org.apache.spark.SparkRuntimeException { "errorClass" : "INVALID_INDEX_OF_ZERO", - "sqlState" : "22003" + "sqlState" : "22003", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "element_at(array(1, 2, 3), 0)" + } ] } -- !query select elt(4, '123', '456') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "2", + "indexValue" : "4" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "elt(4, '123', '456')" + } ] +} -- !query select elt(0, '123', '456') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "2", + "indexValue" : "0" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "elt(0, '123', '456')" + } ] +} -- !query select elt(-1, '123', '456') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "2", + "indexValue" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "elt(-1, '123', '456')" + } ] +} -- !query @@ -262,17 +349,49 @@ NULL -- !query select array(1, 2, 3)[5] -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "3", + "indexValue" : "5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "array(1, 2, 3)[5]" + } ] +} -- !query select array(1, 2, 3)[-1] -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "arraySize" : "3", + "indexValue" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "array(1, 2, 3)[-1]" + } ] +} -- !query @@ -354,7 +473,7 @@ select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) -- !query schema struct -- !query output --1 +NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 738697c638832..0dbdf1d9975c9 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -2,113 +2,337 @@ -- !query SELECT CAST('1.23' AS int) -- !query schema -struct +struct<> -- !query output -1 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "CAST('1.23' AS int)" + } ] +} -- !query SELECT CAST('1.23' AS long) -- !query schema -struct +struct<> -- !query output -1 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('1.23' AS long)" + } ] +} -- !query SELECT CAST('-4.56' AS int) -- !query schema -struct +struct<> -- !query output --4 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'-4.56'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('-4.56' AS int)" + } ] +} -- !query SELECT CAST('-4.56' AS long) -- !query schema -struct +struct<> -- !query output --4 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'-4.56'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('-4.56' AS long)" + } ] +} -- !query SELECT CAST('abc' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "CAST('abc' AS int)" + } ] +} -- !query SELECT CAST('abc' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "CAST('abc' AS long)" + } ] +} -- !query SELECT CAST('abc' AS float) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('abc' AS float)" + } ] +} -- !query SELECT CAST('abc' AS double) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('abc' AS double)" + } ] +} -- !query SELECT CAST('1234567890123' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "CAST('1234567890123' AS int)" + } ] +} -- !query SELECT CAST('12345678901234567890123' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'12345678901234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "CAST('12345678901234567890123' AS long)" + } ] +} -- !query SELECT CAST('' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CAST('' AS int)" + } ] +} -- !query SELECT CAST('' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "CAST('' AS long)" + } ] +} -- !query SELECT CAST('' AS float) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "CAST('' AS float)" + } ] +} -- !query SELECT CAST('' AS double) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "CAST('' AS double)" + } ] +} -- !query @@ -130,33 +354,97 @@ NULL -- !query SELECT CAST('123.a' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('123.a' AS int)" + } ] +} -- !query SELECT CAST('123.a' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('123.a' AS long)" + } ] +} -- !query SELECT CAST('123.a' AS float) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "CAST('123.a' AS float)" + } ] +} -- !query SELECT CAST('123.a' AS double) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "CAST('123.a' AS double)" + } ] +} -- !query @@ -170,9 +458,25 @@ struct -- !query SELECT CAST('-2147483649' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'-2147483649'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "CAST('-2147483649' AS int)" + } ] +} -- !query @@ -186,9 +490,25 @@ struct -- !query SELECT CAST('2147483648' AS int) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2147483648'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "CAST('2147483648' AS int)" + } ] +} -- !query @@ -202,9 +522,25 @@ struct -- !query SELECT CAST('-9223372036854775809' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'-9223372036854775809'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "CAST('-9223372036854775809' AS long)" + } ] +} -- !query @@ -218,9 +554,25 @@ struct -- !query SELECT CAST('9223372036854775808' AS long) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'9223372036854775808'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "CAST('9223372036854775808' AS long)" + } ] +} -- !query @@ -234,65 +586,209 @@ struct -- !query SELECT HEX(CAST(CAST(123 AS byte) AS binary)) -- !query schema -struct +struct<> -- !query output -7B +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "CAST(CAST(123 AS byte) AS binary)" + } ] +} -- !query SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) -- !query schema -struct +struct<> -- !query output -85 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 45, + "fragment" : "CAST(CAST(-123 AS byte) AS binary)" + } ] +} -- !query SELECT HEX(CAST(123S AS binary)) -- !query schema -struct +struct<> -- !query output -007B +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123S AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123S AS binary)) -- !query schema -struct +struct<> -- !query output -FF85 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123S AS binary)" + } ] +} -- !query SELECT HEX(CAST(123 AS binary)) -- !query schema -struct +struct<> -- !query output -0000007B +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 30, + "fragment" : "CAST(123 AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123 AS binary)) -- !query schema -struct +struct<> -- !query output -FFFFFF85 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(-123 AS binary)" + } ] +} -- !query SELECT HEX(CAST(123L AS binary)) -- !query schema -struct +struct<> -- !query output -000000000000007B +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123L AS binary)" + } ] +} -- !query SELECT HEX(CAST(-123L AS binary)) -- !query schema -struct +struct<> -- !query output -FFFFFFFFFFFFFF85 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123L AS binary)" + } ] +} -- !query @@ -448,41 +944,121 @@ struct -- !query select cast('1中文' as tinyint) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TINYINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast('1中文' as tinyint)" + } ] +} -- !query select cast('1中文' as smallint) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"SMALLINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast('1中文' as smallint)" + } ] +} -- !query select cast('1中文' as INT) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast('1中文' as INT)" + } ] +} -- !query select cast('中文1' as bigint) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'中文1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast('中文1' as bigint)" + } ] +} -- !query select cast('1中文' as bigint) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast('1中文' as bigint)" + } ] +} -- !query @@ -506,10 +1082,25 @@ false -- !query select cast('\t\n xyz \t\r' as boolean) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'\t\n xyz \t\r'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast('\\t\\n xyz \\t\\r' as boolean)" + } ] +} -- !query @@ -523,17 +1114,50 @@ struct -- !query select cast('123.45' as decimal(4, 2)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "4", + "scale" : "2", + "value" : "123.45" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast('123.45' as decimal(4, 2))" + } ] +} -- !query select cast('xyz' as decimal(4, 2)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'xyz'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(4,2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast('xyz' as decimal(4, 2))" + } ] +} -- !query @@ -547,9 +1171,25 @@ struct -- !query select cast('a' as date) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast('a' as date)" + } ] +} -- !query @@ -563,9 +1203,25 @@ struct -- !query select cast('a' as timestamp) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast('a' as timestamp)" + } ] +} -- !query @@ -579,25 +1235,73 @@ struct -- !query select cast('a' as timestamp_ntz) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP_NTZ\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast('a' as timestamp_ntz)" + } ] +} -- !query select cast(cast('inf' as double) as timestamp) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "Infinity", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(cast('inf' as double) as timestamp)" + } ] +} -- !query select cast(cast('inf' as float) as timestamp) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "Infinity", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(cast('inf' as float) as timestamp)" + } ] +} -- !query @@ -875,7 +1579,14 @@ org.apache.spark.SparkArithmeticException "precision" : "1", "scale" : "0", "value" : "10.123000" - } + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(interval '10.123' second as decimal(1, 0))" + } ] } @@ -914,33 +1625,97 @@ struct -- !query SELECT '1.23' :: int -- !query schema -struct +struct<> -- !query output -1 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "'1.23' :: int" + } ] +} -- !query SELECT 'abc' :: int -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "'abc' :: int" + } ] +} -- !query SELECT '12345678901234567890123' :: long -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'12345678901234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "'12345678901234567890123' :: long" + } ] +} -- !query SELECT '' :: int -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'' :: int" + } ] +} -- !query @@ -954,9 +1729,25 @@ NULL -- !query SELECT '123.a' :: int -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "'123.a' :: int" + } ] +} -- !query @@ -978,9 +1769,27 @@ struct -- !query SELECT HEX((123 :: byte) :: binary) -- !query schema -struct +struct<> -- !query output -7B +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 34, + "fragment" : "(123 :: byte) :: binary" + } ] +} -- !query @@ -1067,25 +1876,59 @@ struct -- !query SELECT '1.23' :: int :: long -- !query schema -struct +struct<> -- !query output -1 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "'1.23' :: int" + } ] +} -- !query SELECT '2147483648' :: long :: int -- !query schema -struct +struct<> -- !query output --2147483648 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "2147483648L" + } +} -- !query SELECT CAST('2147483648' :: long AS int) -- !query schema -struct +struct<> -- !query output --2147483648 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "2147483648L" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/date.sql.out b/sql/core/src/test/resources/sql-tests/results/date.sql.out index c46c200ff026f..aa283d3249617 100644 --- a/sql/core/src/test/resources/sql-tests/results/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/date.sql.out @@ -49,17 +49,33 @@ struct -- !query select make_date(2000, 13, 1) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" + } +} -- !query select make_date(2000, 1, 33) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33" + } +} -- !query @@ -184,9 +200,17 @@ struct -- !query select to_date("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" + } +} -- !query @@ -242,9 +266,16 @@ struct -- !query select next_day("2015-07-23", "xx") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "xx" + } +} -- !query @@ -274,9 +305,25 @@ struct -- !query select next_day("xx", "Mon") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'xx'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "next_day(\"xx\", \"Mon\")" + } ] +} -- !query @@ -418,13 +465,22 @@ select date_add('2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkNumberFormatException { - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "functionName" : "date_add" - } + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_add('2011-11-11', '1.2')" + } ] } @@ -583,13 +639,22 @@ select date_sub(date'2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkNumberFormatException { - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "functionName" : "date_sub" - } + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "date_sub(date'2011-11-11', '1.2')" + } ] } @@ -628,53 +693,17 @@ struct -- !query select date_add('2011-11-11', int_str) from date_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_add('2011-11-11', int_str)" - } ] -} +2011-11-12 -- !query select date_sub('2011-11-11', int_str) from date_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_sub('2011-11-11', int_str)" - } ] -} +2011-11-10 -- !query @@ -754,27 +783,9 @@ struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> -- !query select date '2001-10-01' - '2001-09-28' -- !query schema -struct<> +struct<(DATE '2001-10-01' - 2001-09-28):interval day> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2001-09-28\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "date '2001-10-01' - '2001-09-28'" - } ] -} +3 00:00:00.000000000 -- !query @@ -812,27 +823,9 @@ struct<(date_str - DATE '2001-09-28'):interval day> -- !query select date '2001-09-28' - date_str from date_view -- !query schema -struct<> +struct<(DATE '2001-09-28' - date_str):interval day> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"date_str\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "date '2001-09-28' - date_str" - } ] -} +-3696 00:00:00.000000000 -- !query @@ -846,7 +839,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -871,11 +864,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index 7353df600dd4f..5635196efc2e5 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -49,17 +49,33 @@ struct -- !query select make_date(2000, 13, 1) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" + } +} -- !query select make_date(2000, 1, 33) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33" + } +} -- !query @@ -184,9 +200,17 @@ struct -- !query select to_date("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"02-29\"" + } +} -- !query @@ -242,9 +266,16 @@ struct -- !query select next_day("2015-07-23", "xx") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "xx" + } +} -- !query @@ -274,9 +305,25 @@ struct -- !query select next_day("xx", "Mon") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'xx'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "next_day(\"xx\", \"Mon\")" + } ] +} -- !query @@ -418,13 +465,22 @@ select date_add('2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkNumberFormatException { - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "functionName" : "date_add" - } + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_add('2011-11-11', '1.2')" + } ] } @@ -583,13 +639,22 @@ select date_sub(date'2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkNumberFormatException { - "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", - "sqlState" : "22023", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "functionName" : "date_sub" - } + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "date_sub(date'2011-11-11', '1.2')" + } ] } @@ -628,53 +693,17 @@ struct -- !query select date_add('2011-11-11', int_str) from date_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_add('2011-11-11', int_str)" - } ] -} +2011-11-12 -- !query select date_sub('2011-11-11', int_str) from date_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"int_str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 38, - "fragment" : "date_sub('2011-11-11', int_str)" - } ] -} +2011-11-10 -- !query @@ -754,27 +783,9 @@ struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> -- !query select date '2001-10-01' - '2001-09-28' -- !query schema -struct<> +struct<(DATE '2001-10-01' - 2001-09-28):interval day> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2001-09-28\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 39, - "fragment" : "date '2001-10-01' - '2001-09-28'" - } ] -} +3 00:00:00.000000000 -- !query @@ -812,27 +823,9 @@ struct<(date_str - DATE '2001-09-28'):interval day> -- !query select date '2001-09-28' - date_str from date_view -- !query schema -struct<> +struct<(DATE '2001-09-28' - date_str):interval day> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"date_str\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "date '2001-09-28' - date_str" - } ] -} +-3696 00:00:00.000000000 -- !query @@ -846,7 +839,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -871,11 +864,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" }, "queryContext" : [ { "objectType" : "", @@ -1316,9 +1309,16 @@ struct -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} -- !query @@ -1340,9 +1340,17 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" + } +} -- !query @@ -1364,17 +1372,33 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" + } +} -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" + } +} -- !query @@ -1554,169 +1578,337 @@ struct -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.0\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123UTC\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12345CST\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123456PST\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234567PST\"" + } +} -- !query select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"123456 2019-10-06 10:11:12.123456PST\"" + } +} -- !query select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"223456 2019-10-06 10:11:12.123456PST\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12\"" + } +} -- !query select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" + } +} -- !query select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 10:11\"" + } +} -- !query select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06S10:11:12.12345\"" + } +} -- !query select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"12.12342019-10-06S10:11\"" + } +} -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" + } +} -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" + } +} -- !query select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"12.1234019-10-06S10:11\"" + } +} -- !query @@ -1778,9 +1970,17 @@ struct -- !query select to_timestamp("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"02-29\"" + } +} -- !query @@ -1826,53 +2026,17 @@ struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema -struct<> +struct<(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:10\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" - } ] -} +0 00:00:01.000000000 -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema -struct<> +struct<(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:11\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" - } ] -} +0 00:00:01.000000000 -- !query @@ -1902,53 +2066,17 @@ struct<> -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema -struct<> +struct<(str - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "str - timestamp'2011-11-11 11:11:11'" - } ] -} +0 00:00:00.000000000 -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema -struct<> +struct<(TIMESTAMP '2011-11-11 11:11:11' - str):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "timestamp'2011-11-11 11:11:11' - str" - } ] -} +0 00:00:00.000000000 -- !query @@ -1958,11 +2086,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -1982,11 +2110,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { @@ -2073,9 +2201,17 @@ struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Unparseable date: \"2019-10-06 A\"" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out index fffbb2a4e017f..0708a523900ff 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out @@ -11,33 +11,65 @@ long overflow -- !query select to_timestamp('1', 'yy') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '1' could not be parsed at index 0" + } +} -- !query select to_timestamp('-12', 'yy') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '-12' could not be parsed at index 0" + } +} -- !query select to_timestamp('123', 'yy') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '123' could not be parsed, unparsed text found at index 2" + } +} -- !query select to_timestamp('1', 'yyy') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '1' could not be parsed at index 0" + } +} -- !query @@ -60,97 +92,193 @@ org.apache.spark.SparkUpgradeException -- !query select to_timestamp('366', 'D') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" + } +} -- !query select to_timestamp('9', 'DD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '9' could not be parsed at index 0" + } +} -- !query select to_timestamp('366', 'DD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" + } +} -- !query select to_timestamp('9', 'DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '9' could not be parsed at index 0" + } +} -- !query select to_timestamp('99', 'DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '99' could not be parsed at index 0" + } +} -- !query select to_timestamp('30-365', 'dd-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." + } +} -- !query select to_timestamp('11-365', 'MM-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31." + } +} -- !query select to_timestamp('2019-366', 'yyyy-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" + } +} -- !query select to_timestamp('12-30-365', 'MM-dd-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." + } +} -- !query select to_timestamp('2020-01-365', 'yyyy-dd-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" + } +} -- !query select to_timestamp('2020-10-350', 'yyyy-MM-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" + } +} -- !query select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" + } +} -- !query @@ -164,78 +292,174 @@ struct> -- !query select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} -- !query select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} -- !query select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} -- !query select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} -- !query select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} -- !query select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} -- !query select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} -- !query select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} -- !query select cast("Unparseable" as timestamp) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'Unparseable'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast(\"Unparseable\" as timestamp)" + } ] +} -- !query select cast("Unparseable" as date) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'Unparseable'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(\"Unparseable\" as date)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index 42e603981848e..cb52778c420ae 100644 --- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -10,25 +10,67 @@ struct<> -- !query select a / b from t -- !query schema -struct<(a / b):decimal(8,6)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "a / b" + } ] +} -- !query select a % b from t -- !query schema -struct<(a % b):decimal(1,1)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "a % b" + } ] +} -- !query select pmod(a, b) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "pmod(a, b)" + } ] +} -- !query @@ -121,41 +163,126 @@ struct<(2.35E10 * 1.0):double> -- !query select (5e36BD + 0.1) + 5e36BD -- !query schema -struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "10000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "(5e36BD + 0.1) + 5e36BD" + } ] +} -- !query select (-4e36BD - 0.1) - 7e36BD -- !query schema -struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "-11000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "(-4e36BD - 0.1) - 7e36BD" + } ] +} -- !query select 12345678901234567890.0 * 12345678901234567890.0 -- !query schema -struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "152415787532388367501905199875019052100" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678901234567890.0 * 12345678901234567890.0" + } ] +} -- !query select 1e35BD / 0.1 -- !query schema -struct<(100000000000000000000000000000000000 / 0.1):decimal(38,6)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "6", + "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "1e35BD / 0.1" + } ] +} -- !query select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query schema -struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "0", + "value" : "15241578753238836750190519987501905210000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "1.2345678901234567890E30BD * 1.2345678901234567890E25BD" + } ] +} -- !query @@ -268,14 +395,28 @@ spark.sql.decimalOperations.allowPrecisionLoss false -- !query -select id, a+b, a-b, a*b, a/b from decimals_test order by id +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id -- !query schema -struct +struct<> -- !query output -1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 -2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 -3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 -4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "36", + "value" : "152.358023429667510000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 43, + "fragment" : "a*b" + } ] +} -- !query @@ -324,9 +465,26 @@ struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,36)> -- !query select 10.300000000000000000 * 3.0000000000000000000 -- !query schema -struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,37)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "37", + "value" : "30.9000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "10.300000000000000000 * 3.0000000000000000000" + } ] +} -- !query @@ -340,81 +498,251 @@ struct<(2.35E10 * 1.0):double> -- !query select (5e36BD + 0.1) + 5e36BD -- !query schema -struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "10000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "(5e36BD + 0.1) + 5e36BD" + } ] +} -- !query select (-4e36BD - 0.1) - 7e36BD -- !query schema -struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "-11000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "(-4e36BD - 0.1) - 7e36BD" + } ] +} -- !query select 12345678901234567890.0 * 12345678901234567890.0 -- !query schema -struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "152415787532388367501905199875019052100" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678901234567890.0 * 12345678901234567890.0" + } ] +} -- !query select 1e35BD / 0.1 -- !query schema -struct<(100000000000000000000000000000000000 / 0.1):decimal(38,3)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "3", + "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "1e35BD / 0.1" + } ] +} -- !query select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query schema -struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "0", + "value" : "15241578753238836750190519987501905210000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "1.2345678901234567890E30BD * 1.2345678901234567890E25BD" + } ] +} -- !query select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 -- !query schema -struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,7)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "7", + "value" : "10012345678912345678912345678911.2469067" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 84, + "fragment" : "12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345" + } ] +} -- !query select 123456789123456789.1234567890 * 1.123456789123456789 -- !query schema -struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,28)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "28", + "value" : "138698367904130467.654320988515622620750" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "123456789123456789.1234567890 * 1.123456789123456789" + } ] +} -- !query select 12345678912345.123456789123 / 0.000000012345678 -- !query schema -struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,18)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "18", + "value" : "1000000073899961059796.725866331521039184725213147467478092333" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678912345.123456789123 / 0.000000012345678" + } ] +} -- !query select 1.0123456789012345678901234567890123456e36BD / 0.1 -- !query schema -struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,2)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "10123456789012345678901234567890123456.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e36BD / 0.1" + } ] +} -- !query select 1.0123456789012345678901234567890123456e35BD / 1.0 -- !query schema -struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,3)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "3", + "value" : "101234567890123456789012345678901234.560000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e35BD / 1.0" + } ] +} -- !query @@ -452,9 +780,26 @@ struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)> -- !query select 1.0123456789012345678901234567890123456e31BD / 0.1 -- !query schema -struct<(10123456789012345678901234567890.123456 / 0.1):decimal(38,6)> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "6", + "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e31BD / 0.1" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out index 9bc5fe0cc379e..d6735f620a638 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out @@ -44,11 +44,25 @@ struct SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2 -- !query schema -struct<(i + 1):int,(f / i):decimal(17,15),substring(s, 2, 3):string,extract(year FROM t):int,(d / 2):double,size(a):int> +struct<> -- !query output -43 0.232142857142857 ell 1970 6.685 3 -43 0.318333333333333 est 1970 6.17283945E8 3 -1338 0.000923335826477 h n 2000 21.0 3 +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'42.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 68, + "fragment" : "d / 2" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index d8a9f4c2e11f5..633133ad7e4d6 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -626,27 +626,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT every("true") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"true\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"BOOLEAN\"", - "sqlExpr" : "\"every(true)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "every(\"true\")" - } ] -} +true -- !query 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 a8a0423bdb3e0..4e012df792dea 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 @@ -123,33 +123,97 @@ struct<(INTERVAL '2' YEAR / 2):interval year to month> -- !query select interval 2 second * 'a' -- !query schema -struct<(INTERVAL '02' SECOND * a):interval day to second> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval 2 second * 'a'" + } ] +} -- !query select interval 2 second / 'a' -- !query schema -struct<(INTERVAL '02' SECOND / a):interval day to second> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval 2 second / 'a'" + } ] +} -- !query select interval 2 year * 'a' -- !query schema -struct<(INTERVAL '2' YEAR * a):interval year to month> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval 2 year * 'a'" + } ] +} -- !query select interval 2 year / 'a' -- !query schema -struct<(INTERVAL '2' YEAR / a):interval year to month> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval 2 year / 'a'" + } ] +} -- !query @@ -171,17 +235,49 @@ struct<(INTERVAL '2' YEAR * 2):interval year to month> -- !query select 'a' * interval 2 second -- !query schema -struct<(INTERVAL '02' SECOND * a):interval day to second> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'a' * interval 2 second" + } ] +} -- !query select 'a' * interval 2 year -- !query schema -struct<(INTERVAL '2' YEAR * a):interval year to month> +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'a' * interval 2 year" + } ] +} -- !query @@ -773,9 +869,26 @@ struct -- !query select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "18", + "scale" : "6", + "value" : "1234567890123456789" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)" + } ] +} -- !query @@ -1877,17 +1990,49 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select '4 11:11' - interval '4 22:12' day to minute -- !query schema -struct<4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE:string> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'4 11:11'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "'4 11:11' - interval '4 22:12' day to minute" + } ] +} -- !query select '4 12:12:12' + interval '4 22:12' day to minute -- !query schema -struct<4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE:string> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'4 12:12:12'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "'4 12:12:12' + interval '4 22:12' day to minute" + } ] +} -- !query @@ -1949,17 +2094,49 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select str - interval '4 22:12' day to minute from interval_view -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "str - interval '4 22:12' day to minute" + } ] +} -- !query select str + interval '4 22:12' day to minute from interval_view -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "str + interval '4 22:12' day to minute" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/math.sql.out b/sql/core/src/test/resources/sql-tests/results/math.sql.out index 09f4383933288..fb60a920040e6 100644 --- a/sql/core/src/test/resources/sql-tests/results/math.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/math.sql.out @@ -42,17 +42,49 @@ struct -- !query SELECT round(127y, -1) -- !query schema -struct --- !query output --126 +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "round(127y, -1)" + } ] +} -- !query SELECT round(-128y, -1) -- !query schema -struct +struct<> -- !query output -126 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "round(-128y, -1)" + } ] +} -- !query @@ -98,17 +130,49 @@ struct -- !query SELECT round(32767s, -1) -- !query schema -struct +struct<> -- !query output --32766 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "round(32767s, -1)" + } ] +} -- !query SELECT round(-32768s, -1) -- !query schema -struct +struct<> -- !query output -32766 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "round(-32768s, -1)" + } ] +} -- !query @@ -154,17 +218,49 @@ struct -- !query SELECT round(2147483647, -1) -- !query schema -struct +struct<> -- !query output --2147483646 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "round(2147483647, -1)" + } ] +} -- !query SELECT round(-2147483647, -1) -- !query schema -struct +struct<> -- !query output -2147483646 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "round(-2147483647, -1)" + } ] +} -- !query @@ -210,17 +306,49 @@ struct -- !query SELECT round(9223372036854775807L, -1) -- !query schema -struct +struct<> -- !query output --9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "round(9223372036854775807L, -1)" + } ] +} -- !query SELECT round(-9223372036854775808L, -1) -- !query schema -struct +struct<> -- !query output -9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "round(-9223372036854775808L, -1)" + } ] +} -- !query @@ -266,17 +394,49 @@ struct -- !query SELECT bround(127y, -1) -- !query schema -struct +struct<> -- !query output --126 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "bround(127y, -1)" + } ] +} -- !query SELECT bround(-128y, -1) -- !query schema -struct +struct<> -- !query output -126 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "bround(-128y, -1)" + } ] +} -- !query @@ -322,17 +482,49 @@ struct -- !query SELECT bround(32767s, -1) -- !query schema -struct +struct<> -- !query output --32766 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "bround(32767s, -1)" + } ] +} -- !query SELECT bround(-32768s, -1) -- !query schema -struct +struct<> -- !query output -32766 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "bround(-32768s, -1)" + } ] +} -- !query @@ -378,17 +570,49 @@ struct -- !query SELECT bround(2147483647, -1) -- !query schema -struct +struct<> -- !query output --2147483646 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "bround(2147483647, -1)" + } ] +} -- !query SELECT bround(-2147483647, -1) -- !query schema -struct +struct<> -- !query output -2147483646 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "bround(-2147483647, -1)" + } ] +} -- !query @@ -434,17 +658,49 @@ struct -- !query SELECT bround(9223372036854775807L, -1) -- !query schema -struct +struct<> -- !query output --9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "bround(9223372036854775807L, -1)" + } ] +} -- !query SELECT bround(-9223372036854775808L, -1) -- !query schema -struct +struct<> -- !query output -9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "bround(-9223372036854775808L, -1)" + } ] +} -- !query @@ -474,25 +730,73 @@ struct -- !query SELECT conv('92233720368547758070', 10, 16) -- !query schema -struct +struct<> -- !query output -FFFFFFFFFFFFFFFF +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "conv('92233720368547758070', 10, 16)" + } ] +} -- !query SELECT conv('9223372036854775807', 36, 10) -- !query schema -struct +struct<> -- !query output -18446744073709551615 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "conv('9223372036854775807', 36, 10)" + } ] +} -- !query SELECT conv('-9223372036854775807', 36, 10) -- !query schema -struct +struct<> -- !query output -18446744073709551615 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "conv('-9223372036854775807', 36, 10)" + } ] +} -- !query @@ -570,9 +874,25 @@ struct<(+ 25.5):double> -- !query SELECT POSITIVE("invalid") -- !query schema -struct<(+ invalid):double> --- !query output -NULL +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'invalid'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "POSITIVE(\"invalid\")" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/array.sql.out similarity index 85% rename from sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/array.sql.out index 7394e428091c7..c1330c620acfb 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/array.sql.out @@ -178,49 +178,17 @@ struct +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "3", - "indexValue" : "5" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 36, - "fragment" : "element_at(array(1, 2, 3), 5)" - } ] -} +NULL -- !query select element_at(array(1, 2, 3), -5) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "3", - "indexValue" : "-5" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 37, - "fragment" : "element_at(array(1, 2, 3), -5)" - } ] -} +NULL -- !query @@ -231,87 +199,32 @@ struct<> org.apache.spark.SparkRuntimeException { "errorClass" : "INVALID_INDEX_OF_ZERO", - "sqlState" : "22003", - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 36, - "fragment" : "element_at(array(1, 2, 3), 0)" - } ] + "sqlState" : "22003" } -- !query select elt(4, '123', '456') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "2", - "indexValue" : "4" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "elt(4, '123', '456')" - } ] -} +NULL -- !query select elt(0, '123', '456') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "2", - "indexValue" : "0" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 27, - "fragment" : "elt(0, '123', '456')" - } ] -} +NULL -- !query select elt(-1, '123', '456') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "2", - "indexValue" : "-1" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "elt(-1, '123', '456')" - } ] -} +NULL -- !query @@ -349,49 +262,17 @@ NULL -- !query select array(1, 2, 3)[5] -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "3", - "indexValue" : "5" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "array(1, 2, 3)[5]" - } ] -} +NULL -- !query select array(1, 2, 3)[-1] -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArrayIndexOutOfBoundsException -{ - "errorClass" : "INVALID_ARRAY_INDEX", - "sqlState" : "22003", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "arraySize" : "3", - "indexValue" : "-1" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 25, - "fragment" : "array(1, 2, 3)[-1]" - } ] -} +NULL -- !query @@ -473,7 +354,7 @@ select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) -- !query schema struct -- !query output -NULL +-1 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/cast.sql.out new file mode 100644 index 0000000000000..738697c638832 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/cast.sql.out @@ -0,0 +1,1156 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CAST('1.23' AS int) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CAST('1.23' AS long) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CAST('-4.56' AS int) +-- !query schema +struct +-- !query output +-4 + + +-- !query +SELECT CAST('-4.56' AS long) +-- !query schema +struct +-- !query output +-4 + + +-- !query +SELECT CAST('abc' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('1234567890123' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('12345678901234567890123' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST(NULL AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST(NULL AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('-2147483648' AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT CAST('-2147483649' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('2147483647' AS int) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT CAST('2147483648' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('-9223372036854775808' AS long) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT CAST('-9223372036854775809' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('9223372036854775807' AS long) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT CAST('9223372036854775808' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT HEX(CAST('abc' AS binary)) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX(CAST(CAST(123 AS byte) AS binary)) +-- !query schema +struct +-- !query output +7B + + +-- !query +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) +-- !query schema +struct +-- !query output +85 + + +-- !query +SELECT HEX(CAST(123S AS binary)) +-- !query schema +struct +-- !query output +007B + + +-- !query +SELECT HEX(CAST(-123S AS binary)) +-- !query schema +struct +-- !query output +FF85 + + +-- !query +SELECT HEX(CAST(123 AS binary)) +-- !query schema +struct +-- !query output +0000007B + + +-- !query +SELECT HEX(CAST(-123 AS binary)) +-- !query schema +struct +-- !query output +FFFFFF85 + + +-- !query +SELECT HEX(CAST(123L AS binary)) +-- !query schema +struct +-- !query output +000000000000007B + + +-- !query +SELECT HEX(CAST(-123L AS binary)) +-- !query schema +struct +-- !query output +FFFFFFFFFFFFFF85 + + +-- !query +DESC FUNCTION boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +DESC FUNCTION EXTENDED boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Extended Usage: + No example/argument for boolean. + + Since: 2.0.1 + +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT CAST("interval '3-1' year to month" AS interval year to month) +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) +-- !query schema +struct +-- !query output +3 00:00:01.000000000 + + +-- !query +SELECT CAST(interval 3 month 1 hour AS string) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 3 month 1 hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "interval 3 month 1 hour" + } ] +} + + +-- !query +SELECT CAST(interval 3 year 1 month AS string) +-- !query schema +struct +-- !query output +INTERVAL '3-1' YEAR TO MONTH + + +-- !query +SELECT CAST(interval 3 day 1 second AS string) +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select cast(' 1' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1\t' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as smallint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as INT) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as bigint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as float) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast(' 1 ' as DOUBLE) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast('1.0 ' as DEC) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast('1中文' as tinyint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as smallint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as INT) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('中文1' as bigint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as bigint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('\t\t true \n\r ' as boolean) +-- !query schema +struct +-- !query output +true + + +-- !query +select cast('\t\n false \t\r' as boolean) +-- !query schema +struct +-- !query output +false + + +-- !query +select cast('\t\n xyz \t\r' as boolean) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('23.45' as decimal(4, 2)) +-- !query schema +struct +-- !query output +23.45 + + +-- !query +select cast('123.45' as decimal(4, 2)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('xyz' as decimal(4, 2)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01' as date) +-- !query schema +struct +-- !query output +2022-01-01 + + +-- !query +select cast('a' as date) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp_ntz) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp_ntz) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(cast('inf' as double) as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(cast('inf' as float) as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(interval '1' year as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(interval '-10-2' year to month as smallint) +-- !query schema +struct +-- !query output +-122 + + +-- !query +select cast(interval '1000' month as int) +-- !query schema +struct +-- !query output +1000 + + +-- !query +select cast(interval -'10.123456' second as tinyint) +-- !query schema +struct +-- !query output +-10 + + +-- !query +select cast(interval '23:59:59' hour to second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"INTERVAL HOUR TO SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '23:59:59' HOUR TO SECOND" + } +} + + +-- !query +select cast(interval -'1 02:03:04.123' day to second as int) +-- !query schema +struct +-- !query output +-93784 + + +-- !query +select cast(interval '10' day as bigint) +-- !query schema +struct +-- !query output +10 + + +-- !query +select cast(interval '-1000' month as tinyint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"INTERVAL MONTH\"", + "targetType" : "\"TINYINT\"", + "value" : "INTERVAL '-1000' MONTH" + } +} + + +-- !query +select cast(interval '1000000' second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"INTERVAL SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '1000000' SECOND" + } +} + + +-- !query +select cast(1Y as interval year) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select cast(-122S as interval year to month) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(ym as interval year to month) from values(-122S) as t(ym) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(1000 as interval month) +-- !query schema +struct +-- !query output +83-4 + + +-- !query +select cast(-10L as interval second) +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select cast(100Y as interval hour to second) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(dt as interval hour to second) from values(100Y) as t(dt) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(-1000S as interval day to second) +-- !query schema +struct +-- !query output +-0 00:16:40.000000000 + + +-- !query +select cast(10 as interval day) +-- !query schema +struct +-- !query output +10 00:00:00.000000000 + + +-- !query +select cast(2147483647 as interval year) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"INT\"", + "targetType" : "\"INTERVAL YEAR\"", + "value" : "2147483647" + } +} + + +-- !query +select cast(-9223372036854775808L as interval day) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INTERVAL DAY\"", + "value" : "-9223372036854775808L" + } +} + + +-- !query +select cast(interval '-1' year as decimal(10, 0)) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select cast(interval '1.000001' second as decimal(10, 6)) +-- !query schema +struct +-- !query output +1.000001 + + +-- !query +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) +-- !query schema +struct +-- !query output +90123.1 + + +-- !query +select cast(interval '10.123' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.005' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.01 + + +-- !query +select cast(interval '10.123' second as decimal(5, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.123' second as decimal(1, 0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "1", + "scale" : "0", + "value" : "10.123000" + } +} + + +-- !query +select cast(10.123456BD as interval day to second) +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +select cast(80.654321BD as interval hour to minute) +-- !query schema +struct +-- !query output +0 01:20:00.000000000 + + +-- !query +select cast(-10.123456BD as interval year to month) +-- !query schema +struct +-- !query output +-0-10 + + +-- !query +select cast(10.654321BD as interval month) +-- !query schema +struct +-- !query output +0-11 + + +-- !query +SELECT '1.23' :: int +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 'abc' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '12345678901234567890123' :: long +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT NULL :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '123.a' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '-2147483648' :: int +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT HEX('abc' :: binary) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX((123 :: byte) :: binary) +-- !query schema +struct +-- !query output +7B + + +-- !query +SELECT 'interval 3 month 1 hour' :: interval +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT interval 3 day 1 second :: string +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select ' 1 ' :: DOUBLE +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select '1.0 ' :: DEC +-- !query schema +struct +-- !query output +1 + + +-- !query +select '\t\t true \n\r ' :: boolean +-- !query schema +struct +-- !query output +true + + +-- !query +select '2022-01-01 00:00:00' :: timestamp +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select interval '-10-2' year to month :: smallint +-- !query schema +struct +-- !query output +-122 + + +-- !query +select -10L :: interval second +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select interval '08:11:10.001' hour to second :: decimal(10, 4) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select 10.123456BD :: interval day to second +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +SELECT '1.23' :: int :: long +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT '2147483648' :: long :: int +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT CAST('2147483648' :: long AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT map(1, '123', 2, '456')[1] :: int +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT '2147483648' :: BINT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"BINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 27, + "fragment" : "BINT" + } ] +} + + +-- !query +SELECT '2147483648' :: SELECT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 29, + "fragment" : "SELECT" + } ] +} + + +-- !query +SELECT FALSE IS NOT NULL :: string +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'::'", + "hint" : "" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/conditional-functions.sql.out similarity index 78% rename from sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/conditional-functions.sql.out index aa8a600f87560..33882561f518a 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/conditional-functions.sql.out @@ -138,25 +138,9 @@ NULL NULL NULL NULL NULL 1 NULL -- !query SELECT nullifzero('abc') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "nullifzero('abc')" - } ] -} +abc -- !query @@ -173,25 +157,9 @@ struct +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'abc'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 24, - "fragment" : "zeroifnull('abc')" - } ] -} +abc -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/date.sql.out similarity index 91% rename from sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/date.sql.out index aa283d3249617..c46c200ff026f 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/date.sql.out @@ -49,33 +49,17 @@ struct -- !query select make_date(2000, 13, 1) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", - "sqlState" : "22023", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" - } -} +NULL -- !query select make_date(2000, 1, 33) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", - "sqlState" : "22023", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33" - } -} +NULL -- !query @@ -200,17 +184,9 @@ struct -- !query select to_date("02-29", "MM-dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Invalid date 'February 29' as '1970' is not a leap year" - } -} +NULL -- !query @@ -266,16 +242,9 @@ struct -- !query select next_day("2015-07-23", "xx") -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkIllegalArgumentException -{ - "errorClass" : "ILLEGAL_DAY_OF_WEEK", - "sqlState" : "22009", - "messageParameters" : { - "string" : "xx" - } -} +NULL -- !query @@ -305,25 +274,9 @@ struct -- !query select next_day("xx", "Mon") -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'xx'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DATE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "next_day(\"xx\", \"Mon\")" - } ] -} +NULL -- !query @@ -465,22 +418,13 @@ select date_add('2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.SparkNumberFormatException +org.apache.spark.sql.AnalysisException { - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", "messageParameters" : { - "expression" : "'1.2'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 36, - "fragment" : "date_add('2011-11-11', '1.2')" - } ] + "functionName" : "date_add" + } } @@ -639,22 +583,13 @@ select date_sub(date'2011-11-11', '1.2') -- !query schema struct<> -- !query output -org.apache.spark.SparkNumberFormatException +org.apache.spark.sql.AnalysisException { - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", "messageParameters" : { - "expression" : "'1.2'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 40, - "fragment" : "date_sub(date'2011-11-11', '1.2')" - } ] + "functionName" : "date_sub" + } } @@ -693,17 +628,53 @@ struct -- !query select date_add('2011-11-11', int_str) from date_view -- !query schema -struct +struct<> -- !query output -2011-11-12 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_add('2011-11-11', int_str)" + } ] +} -- !query select date_sub('2011-11-11', int_str) from date_view -- !query schema -struct +struct<> -- !query output -2011-11-10 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_sub('2011-11-11', int_str)" + } ] +} -- !query @@ -783,9 +754,27 @@ struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> -- !query select date '2001-10-01' - '2001-09-28' -- !query schema -struct<(DATE '2001-10-01' - 2001-09-28):interval day> +struct<> -- !query output -3 00:00:00.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "date '2001-10-01' - '2001-09-28'" + } ] +} -- !query @@ -823,9 +812,27 @@ struct<(date_str - DATE '2001-09-28'):interval day> -- !query select date '2001-09-28' - date_str from date_view -- !query schema -struct<(DATE '2001-09-28' - date_str):interval day> +struct<> -- !query output --3696 00:00:00.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "date '2001-09-28' - date_str" + } ] +} -- !query @@ -839,7 +846,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DATE\"", + "inputType" : "\"DOUBLE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" @@ -864,11 +871,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"DATE '2011-11-11'\"", - "inputType" : "\"DATE\"", + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out new file mode 100644 index 0000000000000..fffbb2a4e017f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out @@ -0,0 +1,241 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('294248', 'y') +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select to_timestamp('1', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('-12', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1', 'yyy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1234567', 'yyyyyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyyyyy'" + } +} + + +-- !query +select to_timestamp('366', 'D') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('366', 'DD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('99', 'DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('30-365', 'dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('11-365', 'MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-366', 'yyyy-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('12-30-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-01-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-10-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query schema +struct> +-- !query output +{"date":null} + + +-- !query +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as date) +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/datetime-special.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/datetime-special.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out new file mode 100644 index 0000000000000..8074a7bf2ac6d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out @@ -0,0 +1,195 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) +-- !query schema +struct<> +-- !query output + + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +-- !query output +NULL + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<(100000000000000000000000000000000000 / 0.1):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> +-- !query output +138698367904130467.654320988515622621 + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> +-- !query output +138698367904130467.654320988515622621 + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)> +-- !query output +1000000073899961059796.725866332 + + +-- !query +select 1.0123456789012345678901234567890123456e36BD / 0.1 +-- !query schema +struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e35BD / 1.0 +-- !query schema +struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e34BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e33BD / 1.0 +-- !query schema +struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e32BD / 1.0 +-- !query schema +struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)> +-- !query output +10123456789012345678901234567890.123456 + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 0.1 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 0.1):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 10.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,6)> +-- !query output +1012345678901234567890123456789.012346 + + +-- !query +drop table decimals_test +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-disabled.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-disabled.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out similarity index 66% rename from sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out index 2444c399a87ec..81a98a60590f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out @@ -4,20 +4,14 @@ SELECT 1 FROM "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 25, - "fragment" : "\"not_exist\"" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -26,12 +20,13 @@ USE SCHEMA "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "SCHEMA_NOT_FOUND", - "sqlState" : "42704", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "schemaName" : "`spark_catalog`.`not_exist`" + "error" : "'\"not_exist\"'", + "hint" : "" } } @@ -41,20 +36,14 @@ ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 23, - "fragment" : "\"not_exist\"" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -63,20 +52,14 @@ ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 21, - "fragment" : "not_exist" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -85,20 +68,14 @@ SELECT 1 AS "not_exist" FROM not_exist -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 30, - "stopIndex" : 38, - "fragment" : "not_exist" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -107,20 +84,14 @@ SELECT 1 FROM not_exist AS X("hello") -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "relationName" : "`not_exist`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 23, - "fragment" : "not_exist" - } ] + "error" : "'\"hello\"'", + "hint" : "" + } } @@ -129,21 +100,14 @@ SELECT "not_exist"() -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "UNRESOLVED_ROUTINE", - "sqlState" : "42883", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "routineName" : "`not_exist`", - "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 20, - "fragment" : "\"not_exist\"()" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -152,21 +116,14 @@ SELECT "not_exist".not_exist() -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "UNRESOLVED_ROUTINE", - "sqlState" : "42883", + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", "messageParameters" : { - "routineName" : "`not_exist`.`not_exist`", - "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "\"not_exist\".not_exist()" - } ] + "error" : "'\"not_exist\"'", + "hint" : "" + } } @@ -344,23 +301,9 @@ org.apache.spark.sql.AnalysisException -- !query SELECT "hello" -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", - "sqlState" : "42703", - "messageParameters" : { - "objectName" : "`hello`" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 14, - "fragment" : "\"hello\"" - } ] -} +hello -- !query @@ -368,15 +311,7 @@ CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"hello\"'", - "hint" : "" - } -} + -- !query @@ -384,30 +319,15 @@ DROP VIEW v -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchTableException -{ - "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", - "sqlState" : "42P01", - "messageParameters" : { - "relationName" : "`spark_catalog`.`default`.`v`" - } -} + -- !query SELECT INTERVAL "1" YEAR -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"1\"'", - "hint" : "" - } -} +1-0 -- !query @@ -447,7 +367,15 @@ CREATE SCHEMA "myschema" -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} -- !query @@ -456,15 +384,31 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} -- !query SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") -- !query schema -struct +struct<> -- !query output -1 +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} -- !query @@ -472,7 +416,15 @@ DROP TABLE "myview" -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} -- !query @@ -480,4 +432,12 @@ DROP SCHEMA "myschema" -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/higher-order-functions.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/higher-order-functions.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out similarity index 94% rename from sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out index 4e012df792dea..a8a0423bdb3e0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/interval.sql.out @@ -123,97 +123,33 @@ struct<(INTERVAL '2' YEAR / 2):interval year to month> -- !query select interval 2 second * 'a' -- !query schema -struct<> +struct<(INTERVAL '02' SECOND * a):interval day to second> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "interval 2 second * 'a'" - } ] -} +NULL -- !query select interval 2 second / 'a' -- !query schema -struct<> +struct<(INTERVAL '02' SECOND / a):interval day to second> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "interval 2 second / 'a'" - } ] -} +NULL -- !query select interval 2 year * 'a' -- !query schema -struct<> +struct<(INTERVAL '2' YEAR * a):interval year to month> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "interval 2 year * 'a'" - } ] -} +NULL -- !query select interval 2 year / 'a' -- !query schema -struct<> +struct<(INTERVAL '2' YEAR / a):interval year to month> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "interval 2 year / 'a'" - } ] -} +NULL -- !query @@ -235,49 +171,17 @@ struct<(INTERVAL '2' YEAR * 2):interval year to month> -- !query select 'a' * interval 2 second -- !query schema -struct<> +struct<(INTERVAL '02' SECOND * a):interval day to second> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 30, - "fragment" : "'a' * interval 2 second" - } ] -} +NULL -- !query select 'a' * interval 2 year -- !query schema -struct<> +struct<(INTERVAL '2' YEAR * a):interval year to month> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"DOUBLE\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 28, - "fragment" : "'a' * interval 2 year" - } ] -} +NULL -- !query @@ -869,26 +773,9 @@ struct -- !query select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", - "sqlState" : "22003", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"", - "precision" : "18", - "scale" : "6", - "value" : "1234567890123456789" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 59, - "fragment" : "make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)" - } ] -} +NULL -- !query @@ -1990,49 +1877,17 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select '4 11:11' - interval '4 22:12' day to minute -- !query schema -struct<> +struct<4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE:string> -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'4 11:11'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 51, - "fragment" : "'4 11:11' - interval '4 22:12' day to minute" - } ] -} +NULL -- !query select '4 12:12:12' + interval '4 22:12' day to minute -- !query schema -struct<> +struct<4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE:string> -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'4 12:12:12'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "'4 12:12:12' + interval '4 22:12' day to minute" - } ] -} +NULL -- !query @@ -2094,49 +1949,17 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select str - interval '4 22:12' day to minute from interval_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 45, - "fragment" : "str - interval '4 22:12' day to minute" - } ] -} +NULL -- !query select str + interval '4 22:12' day to minute from interval_view -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'1'", - "sourceType" : "\"STRING\"", - "targetType" : "\"TIMESTAMP\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 45, - "fragment" : "str + interval '4 22:12' day to minute" - } ] -} +NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out similarity index 75% rename from sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out index b2331ec4ab804..a885525028623 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out @@ -7,51 +7,51 @@ struct ADD false AFTER false AGGREGATE false -ALL true +ALL false ALTER false ALWAYS false ANALYZE false -AND true +AND false ANTI false -ANY true +ANY false ANY_VALUE false ARCHIVE false ARRAY false -AS true +AS false ASC false AT false -AUTHORIZATION true +AUTHORIZATION false BEGIN false BETWEEN false BIGINT false BINARY false BINDING false BOOLEAN false -BOTH true +BOTH false BUCKET false BUCKETS false BY false BYTE false CACHE false -CALL true +CALL false CALLED false CASCADE false -CASE true -CAST true +CASE false +CAST false CATALOG false CATALOGS false CHANGE false CHAR false CHARACTER false -CHECK true +CHECK false CLEAR false CLUSTER false CLUSTERED false CODEGEN false -COLLATE true -COLLATION true +COLLATE false +COLLATION false COLLECTION false -COLUMN true +COLUMN false COLUMNS false COMMENT false COMMIT false @@ -60,17 +60,17 @@ COMPACTIONS false COMPENSATION false COMPUTE false CONCATENATE false -CONSTRAINT true +CONSTRAINT false CONTAINS false COST false -CREATE true -CROSS true +CREATE false +CROSS false CUBE false CURRENT false -CURRENT_DATE true -CURRENT_TIME true -CURRENT_TIMESTAMP true -CURRENT_USER true +CURRENT_DATE false +CURRENT_TIME false +CURRENT_TIMESTAMP false +CURRENT_USER false DATA false DATABASE false DATABASES false @@ -97,49 +97,49 @@ DETERMINISTIC false DFS false DIRECTORIES false DIRECTORY false -DISTINCT true +DISTINCT false DISTRIBUTE false DIV false DO false DOUBLE false DROP false -ELSE true -END true -ESCAPE true +ELSE false +END false +ESCAPE false ESCAPED false EVOLUTION false -EXCEPT true +EXCEPT false EXCHANGE false EXCLUDE false -EXECUTE true +EXECUTE false EXISTS false EXPLAIN false EXPORT false EXTENDED false EXTERNAL false EXTRACT false -FALSE true -FETCH true +FALSE false +FETCH false FIELDS false FILEFORMAT false -FILTER true +FILTER false FIRST false FLOAT false FOLLOWING false -FOR true -FOREIGN true +FOR false +FOREIGN false FORMAT false FORMATTED false -FROM true -FULL true +FROM false +FULL false FUNCTION false FUNCTIONS false GENERATED false GLOBAL false -GRANT true -GROUP true +GRANT false +GROUP false GROUPING false -HAVING true +HAVING false HOUR false HOURS false IDENTIFIER false @@ -149,34 +149,34 @@ IGNORE false ILIKE false IMMEDIATE false IMPORT false -IN true +IN false INCLUDE false INCREMENT false INDEX false INDEXES false -INNER true +INNER false INPATH false INPUT false INPUTFORMAT false INSERT false INT false INTEGER false -INTERSECT true +INTERSECT false INTERVAL false -INTO true +INTO false INVOKER false -IS true +IS false ITEMS false ITERATE false -JOIN true +JOIN false KEYS false LANGUAGE false LAST false -LATERAL true +LATERAL false LAZY false -LEADING true +LEADING false LEAVE false -LEFT true +LEFT false LIKE false LIMIT false LINES false @@ -209,26 +209,26 @@ NAMESPACE false NAMESPACES false NANOSECOND false NANOSECONDS false -NATURAL true +NATURAL false NO false NONE false -NOT true -NULL true +NOT false +NULL false NULLS false NUMERIC false OF false -OFFSET true -ON true -ONLY true +OFFSET false +ON false +ONLY false OPTION false OPTIONS false -OR true -ORDER true +OR false +ORDER false OUT false -OUTER true +OUTER false OUTPUTFORMAT false OVER false -OVERLAPS true +OVERLAPS false OVERLAY false OVERWRITE false PARTITION false @@ -239,7 +239,7 @@ PIVOT false PLACING false POSITION false PRECEDING false -PRIMARY true +PRIMARY false PRINCIPALS false PROPERTIES false PURGE false @@ -252,7 +252,7 @@ RECORDREADER false RECORDWRITER false RECOVER false REDUCE false -REFERENCES true +REFERENCES false REFRESH false RENAME false REPAIR false @@ -265,7 +265,7 @@ RESTRICT false RETURN false RETURNS false REVOKE false -RIGHT true +RIGHT false ROLE false ROLES false ROLLBACK false @@ -277,12 +277,12 @@ SCHEMAS false SECOND false SECONDS false SECURITY false -SELECT true +SELECT false SEMI false SEPARATED false SERDE false SERDEPROPERTIES false -SESSION_USER true +SESSION_USER false SET false SETS false SHORT false @@ -290,12 +290,12 @@ SHOW false SINGLE false SKEWED false SMALLINT false -SOME true +SOME false SORT false SORTED false SOURCE false SPECIFIC false -SQL true +SQL false START false STATISTICS false STORED false @@ -307,14 +307,14 @@ SUBSTRING false SYNC false SYSTEM_TIME false SYSTEM_VERSION false -TABLE true +TABLE false TABLES false TABLESAMPLE false TARGET false TBLPROPERTIES false TERMINATED false -THEN true -TIME true +THEN false +TIME false TIMEDIFF false TIMESTAMP false TIMESTAMPADD false @@ -322,9 +322,9 @@ TIMESTAMPDIFF false TIMESTAMP_LTZ false TIMESTAMP_NTZ false TINYINT false -TO true +TO false TOUCH false -TRAILING true +TRAILING false TRANSACTION false TRANSACTIONS false TRANSFORM false @@ -336,17 +336,17 @@ TYPE false UNARCHIVE false UNBOUNDED false UNCACHE false -UNION true -UNIQUE true -UNKNOWN true +UNION false +UNIQUE false +UNKNOWN false UNLOCK false UNPIVOT false UNSET false UNTIL false UPDATE false USE false -USER true -USING true +USER false +USING false VALUES false VAR false VARCHAR false @@ -358,12 +358,12 @@ VIEWS false VOID false WEEK false WEEKS false -WHEN true -WHERE true +WHEN false +WHERE false WHILE false WINDOW false -WITH true -WITHIN true +WITH false +WITHIN false X false YEAR false YEARS false @@ -375,79 +375,4 @@ SELECT keyword from SQL_KEYWORDS() WHERE reserved -- !query schema struct -- !query output -ALL -AND -ANY -AS -AUTHORIZATION -BOTH -CALL -CASE -CAST -CHECK -COLLATE -COLLATION -COLUMN -CONSTRAINT -CREATE -CROSS -CURRENT_DATE -CURRENT_TIME -CURRENT_TIMESTAMP -CURRENT_USER -DISTINCT -ELSE -END -ESCAPE -EXCEPT -EXECUTE -FALSE -FETCH -FILTER -FOR -FOREIGN -FROM -FULL -GRANT -GROUP -HAVING -IN -INNER -INTERSECT -INTO -IS -JOIN -LATERAL -LEADING -LEFT -NATURAL -NOT -NULL -OFFSET -ON -ONLY -OR -ORDER -OUTER -OVERLAPS -PRIMARY -REFERENCES -RIGHT -SELECT -SESSION_USER -SOME -SQL -TABLE -THEN -TIME -TO -TRAILING -UNION -UNIQUE -UNKNOWN -USER -USING -WHEN -WHERE -WITH -WITHIN + diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/literals.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/literals.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/map.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/map.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/math.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/math.sql.out new file mode 100644 index 0000000000000..09f4383933288 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/math.sql.out @@ -0,0 +1,583 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT round(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, -1) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT round(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(127y, -1) +-- !query schema +struct +-- !query output +-126 + + +-- !query +SELECT round(-128y, -1) +-- !query schema +struct +-- !query output +126 + + +-- !query +SELECT round(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(32767s, -1) +-- !query schema +struct +-- !query output +-32766 + + +-- !query +SELECT round(-32768s, -1) +-- !query schema +struct +-- !query output +32766 + + +-- !query +SELECT round(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(2147483647, -1) +-- !query schema +struct +-- !query output +-2147483646 + + +-- !query +SELECT round(-2147483647, -1) +-- !query schema +struct +-- !query output +2147483646 + + +-- !query +SELECT round(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT round(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +9223372036854775806 + + +-- !query +SELECT bround(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, -1) +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT bround(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(127y, -1) +-- !query schema +struct +-- !query output +-126 + + +-- !query +SELECT bround(-128y, -1) +-- !query schema +struct +-- !query output +126 + + +-- !query +SELECT bround(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(32767s, -1) +-- !query schema +struct +-- !query output +-32766 + + +-- !query +SELECT bround(-32768s, -1) +-- !query schema +struct +-- !query output +32766 + + +-- !query +SELECT bround(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(2147483647, -1) +-- !query schema +struct +-- !query output +-2147483646 + + +-- !query +SELECT bround(-2147483647, -1) +-- !query schema +struct +-- !query output +2147483646 + + +-- !query +SELECT bround(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT bround(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +9223372036854775806 + + +-- !query +SELECT conv('100', 2, 10) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT conv(-10, 16, -10) +-- !query schema +struct +-- !query output +-16 + + +-- !query +SELECT conv('9223372036854775808', 10, 16) +-- !query schema +struct +-- !query output +8000000000000000 + + +-- !query +SELECT conv('92233720368547758070', 10, 16) +-- !query schema +struct +-- !query output +FFFFFFFFFFFFFFFF + + +-- !query +SELECT conv('9223372036854775807', 36, 10) +-- !query schema +struct +-- !query output +18446744073709551615 + + +-- !query +SELECT conv('-9223372036854775807', 36, 10) +-- !query schema +struct +-- !query output +18446744073709551615 + + +-- !query +SELECT BIN(0) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT BIN(25) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25L) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25.5) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT POSITIVE(0Y) +-- !query schema +struct<(+ 0):tinyint> +-- !query output +0 + + +-- !query +SELECT POSITIVE(25) +-- !query schema +struct<(+ 25):int> +-- !query output +25 + + +-- !query +SELECT POSITIVE(-25L) +-- !query schema +struct<(+ -25):bigint> +-- !query output +-25 + + +-- !query +SELECT POSITIVE(25.5) +-- !query schema +struct<(+ 25.5):decimal(3,1)> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("25.5") +-- !query schema +struct<(+ 25.5):double> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("invalid") +-- !query schema +struct<(+ invalid):double> +-- !query output +NULL + + +-- !query +SELECT POSITIVE(null) +-- !query schema +struct<(+ NULL):double> +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/parse-schema-string.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/parse-schema-string.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/string-functions.sql.out similarity index 96% rename from sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/string-functions.sql.out index 706673606625b..3f9f24f817f2c 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/string-functions.sql.out @@ -94,25 +94,9 @@ NULL -- !query select left("abcd", -2), left("abcd", 0), left("abcd", 'a') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 43, - "stopIndex" : 59, - "fragment" : "left(\"abcd\", 'a')" - } ] -} + NULL -- !query @@ -134,25 +118,9 @@ NULL -- !query select right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'a'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 45, - "stopIndex" : 62, - "fragment" : "right(\"abcd\", 'a')" - } ] -} + NULL -- !query @@ -498,49 +466,17 @@ bar -- !query SELECT lpad('hi', 'invalid_length') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'invalid_length'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "lpad('hi', 'invalid_length')" - } ] -} +NULL -- !query SELECT rpad('hi', 'invalid_length') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'invalid_length'", - "sourceType" : "\"STRING\"", - "targetType" : "\"INT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 35, - "fragment" : "rpad('hi', 'invalid_length')" - } ] -} +NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out similarity index 89% rename from sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out index e3cf1a1549228..0e0b014a3b161 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/timestamp.sql.out @@ -119,16 +119,9 @@ struct -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "INVALID_FRACTION_OF_SECOND", - "sqlState" : "22023", - "messageParameters" : { - "secAndMicros" : "60.007" - } -} +NULL -- !query @@ -150,17 +143,9 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", - "sqlState" : "22023", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" - } -} +NULL -- !query @@ -182,33 +167,17 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", - "sqlState" : "22023", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" - } -} +NULL -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", - "sqlState" : "22023", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" - } -} +NULL -- !query @@ -388,17 +357,9 @@ struct -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" - } -} +NULL -- !query @@ -460,17 +421,9 @@ struct +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" - } -} +NULL -- !query @@ -484,17 +437,9 @@ struct +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" - } -} +NULL -- !query @@ -556,33 +501,17 @@ struct +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" - } -} +NULL -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" - } -} +NULL -- !query @@ -652,17 +581,9 @@ struct -- !query select to_timestamp("02-29", "MM-dd") -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkDateTimeException -{ - "errorClass" : "CANNOT_PARSE_TIMESTAMP", - "sqlState" : "22007", - "messageParameters" : { - "ansiConfig" : "\"spark.sql.ansi.enabled\"", - "message" : "Invalid date 'February 29' as '1970' is not a leap year" - } -} +NULL -- !query @@ -708,17 +629,53 @@ struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema -struct<(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> +struct<> -- !query output -0 00:00:01.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" + } ] +} -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema -struct<(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +struct<> -- !query output -0 00:00:01.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" + } ] +} -- !query @@ -748,17 +705,53 @@ struct<> -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema -struct<(str - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +struct<> -- !query output -0 00:00:00.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "str - timestamp'2011-11-11 11:11:11'" + } ] +} -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema -struct<(TIMESTAMP '2011-11-11 11:11:11' - str):interval day to second> +struct<> -- !query output -0 00:00:00.000000000 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} -- !query @@ -768,11 +761,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "actualDataType" : "\"TIMESTAMP\"", - "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -792,11 +785,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", "sqlState" : "42K09", "messageParameters" : { - "actualDataType" : "\"TIMESTAMP\"", - "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out similarity index 67% rename from sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out index 94048ac8897bb..df1fe996781ad 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out @@ -82,91 +82,33 @@ NULL -- !query SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 23, - "fragment" : "col + 1L" - } ] -} +-9223372036854775806 -- !query @@ -290,91 +232,33 @@ NULL -- !query SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 22, - "fragment" : "col / 0" - } ] -} +NULL -- !query SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 16, - "stopIndex" : 23, - "fragment" : "col + 1L" - } ] -} +-4.6116860184273879E18 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out similarity index 66% rename from sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out index acf6e70a50dea..b12680c2a6751 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out @@ -26,9 +26,9 @@ struct -- !query SELECT try_add(2147483647, "1") -- !query schema -struct +struct -- !query output -2147483648 +2.147483648E9 -- !query @@ -58,71 +58,25 @@ NULL -- !query SELECT try_add(1, (2147483647 + 1)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 33, - "fragment" : "2147483647 + 1" - } ] -} +-2147483647 -- !query SELECT try_add(1L, (9223372036854775807L + 1L)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 21, - "stopIndex" : 45, - "fragment" : "9223372036854775807L + 1L" - } ] -} +-9223372036854775807 -- !query SELECT try_add(1, 1.0 / 0.0) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 27, - "fragment" : "1.0 / 0.0" - } ] -} +NULL -- !query @@ -290,71 +244,25 @@ NULL -- !query SELECT try_divide(1, (2147483647 + 1)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 23, - "stopIndex" : 36, - "fragment" : "2147483647 + 1" - } ] -} +-4.6566128730773926E-10 -- !query SELECT try_divide(1L, (9223372036854775807L + 1L)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 48, - "fragment" : "9223372036854775807L + 1L" - } ] -} +-1.0842021724855044E-19 -- !query SELECT try_divide(1, 1.0 / 0.0) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 22, - "stopIndex" : 30, - "fragment" : "1.0 / 0.0" - } ] -} +NULL -- !query @@ -448,9 +356,9 @@ struct -- !query SELECT try_subtract(2147483647, "-1") -- !query schema -struct +struct -- !query output -2147483648 +2.147483648E9 -- !query @@ -480,71 +388,25 @@ NULL -- !query SELECT try_subtract(1, (2147483647 + 1)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 25, - "stopIndex" : 38, - "fragment" : "2147483647 + 1" - } ] -} +NULL -- !query SELECT try_subtract(1L, (9223372036854775807L + 1L)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 50, - "fragment" : "9223372036854775807L + 1L" - } ] -} +NULL -- !query SELECT try_subtract(1, 1.0 / 0.0) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 32, - "fragment" : "1.0 / 0.0" - } ] -} +NULL -- !query @@ -606,9 +468,9 @@ struct -- !query SELECT try_multiply(2147483647, "-2") -- !query schema -struct +struct -- !query output --4294967294 +-4.294967294E9 -- !query @@ -638,71 +500,25 @@ NULL -- !query SELECT try_multiply(1, (2147483647 + 1)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 25, - "stopIndex" : 38, - "fragment" : "2147483647 + 1" - } ] -} +-2147483648 -- !query SELECT try_multiply(1L, (9223372036854775807L + 1L)) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 50, - "fragment" : "9223372036854775807L + 1L" - } ] -} +-9223372036854775808 -- !query SELECT try_multiply(1, 1.0 / 0.0) -- !query schema -struct<> +struct -- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 32, - "fragment" : "1.0 / 0.0" - } ] -} +NULL -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/try_datetime_functions.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/try_datetime_functions.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/try_element_at.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/try_element_at.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/operators.sql.out b/sql/core/src/test/resources/sql-tests/results/operators.sql.out index 93ac6b49b0168..356e5eca5feb2 100644 --- a/sql/core/src/test/resources/sql-tests/results/operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/operators.sql.out @@ -130,9 +130,23 @@ struct<(5 / 2):double> -- !query select 5 / 0 -- !query schema -struct<(5 / 0):double> --- !query output -NULL +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "5 / 0" + } ] +} -- !query @@ -162,9 +176,23 @@ struct<(5 div 2):bigint> -- !query select 5 div 0 -- !query schema -struct<(5 div 0):bigint> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "5 div 0" + } ] +} -- !query @@ -194,9 +222,23 @@ struct<(CAST(51 AS DECIMAL(10,0)) div CAST(2 AS DECIMAL(2,0))):bigint> -- !query select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)) -- !query schema -struct<(CAST(5 AS DECIMAL(1,0)) div CAST(0 AS DECIMAL(2,0))):bigint> +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0))" + } ] +} -- !query @@ -450,9 +492,23 @@ true -- !query select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) -- !query schema -struct +struct<> -- !query output -1 NULL 0 NULL NULL NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "mod(7, 0)" + } ] +} -- !query @@ -506,17 +562,45 @@ struct<(+ -1.11):double,(+ -1.11):decimal(3,2),negative(-1.11):double,negative(- -- !query select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) -- !query schema -struct +struct<> -- !query output -1 0 NULL NULL NULL NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 42, + "fragment" : "pmod(7, 0)" + } ] +} -- !query select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) -- !query schema -struct --- !query output -NULL NULL +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "pmod(cast(3.13 as decimal), cast(0 as decimal))" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index aae68dddbaab3..a365e759b7c14 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -1581,12 +1581,27 @@ struct -- !query values (0, 1) tab(x, y) |> union table t +|> where x = 0 -- !query schema -struct +struct<> -- !query output -0 1 -0 abc -1 def +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 55, + "fragment" : "values (0, 1) tab(x, y)\n|> union table t\n|> where x = 0" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out index 5b97f2a27b8ed..e2d0563a0c451 100644 --- a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out @@ -210,25 +210,73 @@ false -- !query select 2 > '1.0' -- !query schema -struct<(2 > 1.0):boolean> --- !query output -true +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '1.0'" + } ] +} -- !query select 2 > '2.0' -- !query schema -struct<(2 > 2.0):boolean> +struct<> -- !query output -false +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '2.0'" + } ] +} -- !query select 2 > '2.2' -- !query schema -struct<(2 > 2.2):boolean> +struct<> -- !query output -false +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '2.2'" + } ] +} -- !query @@ -266,17 +314,49 @@ true -- !query select 2 >= '1.0' -- !query schema -struct<(2 >= 1.0):boolean> +struct<> -- !query output -true +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 >= '1.0'" + } ] +} -- !query select 2 >= '2.0' -- !query schema -struct<(2 >= 2.0):boolean> +struct<> -- !query output -true +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 >= '2.0'" + } ] +} -- !query @@ -322,17 +402,49 @@ false -- !query select 2 < '1.0' -- !query schema -struct<(2 < 1.0):boolean> +struct<> -- !query output -false +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 < '1.0'" + } ] +} -- !query select 2 < '2.0' -- !query schema -struct<(2 < 2.0):boolean> +struct<> -- !query output -false +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 < '2.0'" + } ] +} -- !query @@ -378,17 +490,49 @@ true -- !query select 2 <= '1.0' -- !query schema -struct<(2 <= 1.0):boolean> +struct<> -- !query output -false +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 <= '1.0'" + } ] +} -- !query select 2 <= '2.0' -- !query schema -struct<(2 <= 2.0):boolean> --- !query output -true +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 <= '2.0'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index 3f9f24f817f2c..706673606625b 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -94,9 +94,25 @@ NULL -- !query select left("abcd", -2), left("abcd", 0), left("abcd", 'a') -- !query schema -struct +struct<> -- !query output - NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 59, + "fragment" : "left(\"abcd\", 'a')" + } ] +} -- !query @@ -118,9 +134,25 @@ NULL -- !query select right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query schema -struct +struct<> -- !query output - NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 62, + "fragment" : "right(\"abcd\", 'a')" + } ] +} -- !query @@ -466,17 +498,49 @@ bar -- !query SELECT lpad('hi', 'invalid_length') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'invalid_length'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "lpad('hi', 'invalid_length')" + } ] +} -- !query SELECT rpad('hi', 'invalid_length') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'invalid_length'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "rpad('hi', 'invalid_length')" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/subexp-elimination.sql.out b/sql/core/src/test/resources/sql-tests/results/subexp-elimination.sql.out index 0f7ff3f107567..e355055895162 100644 --- a/sql/core/src/test/resources/sql-tests/results/subexp-elimination.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subexp-elimination.sql.out @@ -48,12 +48,12 @@ NULL -- !query SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 else from_json(a, 'struct').b + 2 end FROM testData -- !query schema -struct 5) THEN from_json(a).b WHEN (from_json(a).a > 4) THEN (from_json(a).b + 1) ELSE (from_json(a).b + 2) END:string> +struct 5) THEN from_json(a).b WHEN (from_json(a).a > 4) THEN (from_json(a).b + 1) ELSE (from_json(a).b + 2) END:bigint> -- !query output -4.0 -4.0 -5.0 -7.0 +4 +4 +5 +7 NULL diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ltz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ltz.sql.out index b4ab5bdeb4ff8..963505615225a 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ltz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ltz.sql.out @@ -50,9 +50,16 @@ struct -- !query SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out index 81fa1f1dc3327..3a473dad828a9 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out @@ -67,9 +67,16 @@ org.apache.spark.sql.AnalysisException -- !query SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 60.007) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out index 0e0b014a3b161..e3cf1a1549228 100644 --- a/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timestamp.sql.out @@ -119,9 +119,16 @@ struct -- !query SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} -- !query @@ -143,9 +150,17 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 61) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" + } +} -- !query @@ -167,17 +182,33 @@ struct -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" + } +} -- !query SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" + } +} -- !query @@ -357,9 +388,17 @@ struct -- !query select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" + } +} -- !query @@ -421,9 +460,17 @@ struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" + } +} -- !query @@ -437,9 +484,17 @@ struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" + } +} -- !query @@ -501,17 +556,33 @@ struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" + } +} -- !query select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" + } +} -- !query @@ -581,9 +652,17 @@ struct -- !query select to_timestamp("02-29", "MM-dd") -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" + } +} -- !query @@ -629,53 +708,17 @@ struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day -- !query select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' -- !query schema -struct<> +struct<(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:10\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" - } ] -} +0 00:00:01.000000000 -- !query select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' -- !query schema -struct<> +struct<(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"2011-11-11 11:11:11\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 61, - "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" - } ] -} +0 00:00:01.000000000 -- !query @@ -705,53 +748,17 @@ struct<> -- !query select str - timestamp'2011-11-11 11:11:11' from ts_view -- !query schema -struct<> +struct<(str - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "str - timestamp'2011-11-11 11:11:11'" - } ] -} +0 00:00:00.000000000 -- !query select timestamp'2011-11-11 11:11:11' - str from ts_view -- !query schema -struct<> +struct<(TIMESTAMP '2011-11-11 11:11:11' - str):interval day to second> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"str\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 43, - "fragment" : "timestamp'2011-11-11 11:11:11' - str" - } ] -} +0 00:00:00.000000000 -- !query @@ -761,11 +768,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" }, "queryContext" : [ { @@ -785,11 +792,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" }, "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/results/transform.sql.out b/sql/core/src/test/resources/sql-tests/results/transform.sql.out index 7975392fd0147..3c704dfcc4618 100644 --- a/sql/core/src/test/resources/sql-tests/results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/transform.sql.out @@ -611,11 +611,11 @@ FROM( SELECT (b + 1) AS result ORDER BY result -- !query schema -struct +struct -- !query output -3.0 -6.0 -9.0 +3 +6 +9 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out b/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out index df1fe996781ad..94048ac8897bb 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out @@ -82,33 +82,91 @@ NULL -- !query SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct +struct<> -- !query output --9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} -- !query @@ -232,33 +290,91 @@ NULL -- !query SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct +struct<> -- !query output --4.6116860184273879E18 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out index b12680c2a6751..acf6e70a50dea 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out @@ -26,9 +26,9 @@ struct -- !query SELECT try_add(2147483647, "1") -- !query schema -struct +struct -- !query output -2.147483648E9 +2147483648 -- !query @@ -58,25 +58,71 @@ NULL -- !query SELECT try_add(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --2147483647 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 33, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_add(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --9223372036854775807 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 45, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_add(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -244,25 +290,71 @@ NULL -- !query SELECT try_divide(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --4.6566128730773926E-10 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 36, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_divide(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --1.0842021724855044E-19 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 48, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_divide(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 30, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -356,9 +448,9 @@ struct -- !query SELECT try_subtract(2147483647, "-1") -- !query schema -struct +struct -- !query output -2.147483648E9 +2147483648 -- !query @@ -388,25 +480,71 @@ NULL -- !query SELECT try_subtract(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 38, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_subtract(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_subtract(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -468,9 +606,9 @@ struct -- !query SELECT try_multiply(2147483647, "-2") -- !query schema -struct +struct -- !query output --4.294967294E9 +-4294967294 -- !query @@ -500,25 +638,71 @@ NULL -- !query SELECT try_multiply(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --2147483648 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 38, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_multiply(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --9223372036854775808 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_multiply(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out index ee0536967ad3d..f3263241a5561 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out @@ -10,193 +10,625 @@ struct<> -- !query SELECT cast(1 as binary) = '1' FROM t -- !query schema -struct<(CAST(1 AS BINARY) = 1):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) > '2' FROM t -- !query schema -struct<(CAST(1 AS BINARY) > 2):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) >= '2' FROM t -- !query schema -struct<(CAST(1 AS BINARY) >= 2):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) < '2' FROM t -- !query schema -struct<(CAST(1 AS BINARY) < 2):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <= '2' FROM t -- !query schema -struct<(CAST(1 AS BINARY) <= 2):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <> '2' FROM t -- !query schema -struct<(NOT (CAST(1 AS BINARY) = 2)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) = cast(null as string) FROM t -- !query schema -struct<(CAST(1 AS BINARY) = CAST(NULL AS STRING)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) > cast(null as string) FROM t -- !query schema -struct<(CAST(1 AS BINARY) > CAST(NULL AS STRING)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) >= cast(null as string) FROM t -- !query schema -struct<(CAST(1 AS BINARY) >= CAST(NULL AS STRING)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) < cast(null as string) FROM t -- !query schema -struct<(CAST(1 AS BINARY) < CAST(NULL AS STRING)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <= cast(null as string) FROM t -- !query schema -struct<(CAST(1 AS BINARY) <= CAST(NULL AS STRING)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(1 as binary) <> cast(null as string) FROM t -- !query schema -struct<(NOT (CAST(1 AS BINARY) = CAST(NULL AS STRING))):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '1' = cast(1 as binary) FROM t -- !query schema -struct<(1 = CAST(1 AS BINARY)):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' > cast(1 as binary) FROM t -- !query schema -struct<(2 > CAST(1 AS BINARY)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' >= cast(1 as binary) FROM t -- !query schema -struct<(2 >= CAST(1 AS BINARY)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' < cast(1 as binary) FROM t -- !query schema -struct<(2 < CAST(1 AS BINARY)):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' <= cast(1 as binary) FROM t -- !query schema -struct<(2 <= CAST(1 AS BINARY)):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT '2' <> cast(1 as binary) FROM t -- !query schema -struct<(NOT (2 = CAST(1 AS BINARY))):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) = cast(1 as binary) FROM t -- !query schema -struct<(CAST(NULL AS STRING) = CAST(1 AS BINARY)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) > cast(1 as binary) FROM t -- !query schema -struct<(CAST(NULL AS STRING) > CAST(1 AS BINARY)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) >= cast(1 as binary) FROM t -- !query schema -struct<(CAST(NULL AS STRING) >= CAST(1 AS BINARY)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) < cast(1 as binary) FROM t -- !query schema -struct<(CAST(NULL AS STRING) < CAST(1 AS BINARY)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) <= cast(1 as binary) FROM t -- !query schema -struct<(CAST(NULL AS STRING) <= CAST(1 AS BINARY)):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query SELECT cast(null as string) <> cast(1 as binary) FROM t -- !query schema -struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS BINARY))):boolean> +struct<> -- !query output -NULL +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out index 9b363b15c507f..f83284e9bf6f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/booleanEquality.sql.out @@ -10,57 +10,169 @@ struct<> -- !query SELECT true = cast(1 as tinyint) FROM t -- !query schema -struct<(true = CAST(1 AS TINYINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true = CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "true = cast(1 as tinyint)" + } ] +} -- !query SELECT true = cast(1 as smallint) FROM t -- !query schema -struct<(true = CAST(1 AS SMALLINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true = CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true = cast(1 as smallint)" + } ] +} -- !query SELECT true = cast(1 as int) FROM t -- !query schema -struct<(true = CAST(1 AS INT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true = CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "true = cast(1 as int)" + } ] +} -- !query SELECT true = cast(1 as bigint) FROM t -- !query schema -struct<(true = CAST(1 AS BIGINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true = CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as bigint)" + } ] +} -- !query SELECT true = cast(1 as float) FROM t -- !query schema -struct<(true = CAST(1 AS FLOAT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true = CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true = cast(1 as float)" + } ] +} -- !query SELECT true = cast(1 as double) FROM t -- !query schema -struct<(true = CAST(1 AS DOUBLE)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true = CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as double)" + } ] +} -- !query SELECT true = cast(1 as decimal(10, 0)) FROM t -- !query schema -struct<(true = CAST(1 AS DECIMAL(10,0))):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "true = cast(1 as decimal(10, 0))" + } ] +} -- !query @@ -154,57 +266,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT true <=> cast(1 as tinyint) FROM t -- !query schema -struct<(true <=> CAST(1 AS TINYINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "true <=> cast(1 as tinyint)" + } ] +} -- !query SELECT true <=> cast(1 as smallint) FROM t -- !query schema -struct<(true <=> CAST(1 AS SMALLINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "true <=> cast(1 as smallint)" + } ] +} -- !query SELECT true <=> cast(1 as int) FROM t -- !query schema -struct<(true <=> CAST(1 AS INT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true <=> cast(1 as int)" + } ] +} -- !query SELECT true <=> cast(1 as bigint) FROM t -- !query schema -struct<(true <=> CAST(1 AS BIGINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as bigint)" + } ] +} -- !query SELECT true <=> cast(1 as float) FROM t -- !query schema -struct<(true <=> CAST(1 AS FLOAT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "true <=> cast(1 as float)" + } ] +} -- !query SELECT true <=> cast(1 as double) FROM t -- !query schema -struct<(true <=> CAST(1 AS DOUBLE)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as double)" + } ] +} -- !query SELECT true <=> cast(1 as decimal(10, 0)) FROM t -- !query schema -struct<(true <=> CAST(1 AS DECIMAL(10,0))):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "true <=> cast(1 as decimal(10, 0))" + } ] +} -- !query @@ -298,57 +522,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) = true FROM t -- !query schema -struct<(CAST(1 AS TINYINT) = true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as tinyint) = true" + } ] +} -- !query SELECT cast(1 as smallint) = true FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) = true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as smallint) = true" + } ] +} -- !query SELECT cast(1 as int) = true FROM t -- !query schema -struct<(CAST(1 AS INT) = true):boolean> --- !query output -true - - --- !query -SELECT cast(1 as bigint) = true FROM t --- !query schema -struct<(CAST(1 AS BIGINT) = true):boolean> +struct<> -- !query output -true - +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast(1 as int) = true" + } ] +} + + +-- !query +SELECT cast(1 as bigint) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as bigint) = true" + } ] +} + -- !query SELECT cast(1 as float) = true FROM t -- !query schema -struct<(CAST(1 AS FLOAT) = true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as float) = true" + } ] +} -- !query SELECT cast(1 as double) = true FROM t -- !query schema -struct<(CAST(1 AS DOUBLE) = true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as double) = true" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) = true FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(10,0)) = true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast(1 as decimal(10, 0)) = true" + } ] +} -- !query @@ -442,57 +778,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) <=> true FROM t -- !query schema -struct<(CAST(1 AS TINYINT) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(1 as tinyint) <=> true" + } ] +} -- !query SELECT cast(1 as smallint) <=> true FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(1 as smallint) <=> true" + } ] +} -- !query SELECT cast(1 as int) <=> true FROM t -- !query schema -struct<(CAST(1 AS INT) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as int) <=> true" + } ] +} -- !query SELECT cast(1 as bigint) <=> true FROM t -- !query schema -struct<(CAST(1 AS BIGINT) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as bigint) <=> true" + } ] +} -- !query SELECT cast(1 as float) <=> true FROM t -- !query schema -struct<(CAST(1 AS FLOAT) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as float) <=> true" + } ] +} -- !query SELECT cast(1 as double) <=> true FROM t -- !query schema -struct<(CAST(1 AS DOUBLE) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as double) <=> true" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <=> true FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(10,0)) <=> true):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as decimal(10, 0)) <=> true" + } ] +} -- !query @@ -586,57 +1034,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT false = cast(0 as tinyint) FROM t -- !query schema -struct<(false = CAST(0 AS TINYINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false = CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false = cast(0 as tinyint)" + } ] +} -- !query SELECT false = cast(0 as smallint) FROM t -- !query schema -struct<(false = CAST(0 AS SMALLINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false = CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false = cast(0 as smallint)" + } ] +} -- !query SELECT false = cast(0 as int) FROM t -- !query schema -struct<(false = CAST(0 AS INT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false = CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "false = cast(0 as int)" + } ] +} -- !query SELECT false = cast(0 as bigint) FROM t -- !query schema -struct<(false = CAST(0 AS BIGINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false = CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as bigint)" + } ] +} -- !query SELECT false = cast(0 as float) FROM t -- !query schema -struct<(false = CAST(0 AS FLOAT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false = CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false = cast(0 as float)" + } ] +} -- !query SELECT false = cast(0 as double) FROM t -- !query schema -struct<(false = CAST(0 AS DOUBLE)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false = CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as double)" + } ] +} -- !query SELECT false = cast(0 as decimal(10, 0)) FROM t -- !query schema -struct<(false = CAST(0 AS DECIMAL(10,0))):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false = CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "false = cast(0 as decimal(10, 0))" + } ] +} -- !query @@ -730,57 +1290,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT false <=> cast(0 as tinyint) FROM t -- !query schema -struct<(false <=> CAST(0 AS TINYINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "false <=> cast(0 as tinyint)" + } ] +} -- !query SELECT false <=> cast(0 as smallint) FROM t -- !query schema -struct<(false <=> CAST(0 AS SMALLINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast(0 as smallint)" + } ] +} -- !query SELECT false <=> cast(0 as int) FROM t -- !query schema -struct<(false <=> CAST(0 AS INT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false <=> cast(0 as int)" + } ] +} -- !query SELECT false <=> cast(0 as bigint) FROM t -- !query schema -struct<(false <=> CAST(0 AS BIGINT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as bigint)" + } ] +} -- !query SELECT false <=> cast(0 as float) FROM t -- !query schema -struct<(false <=> CAST(0 AS FLOAT)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false <=> cast(0 as float)" + } ] +} -- !query SELECT false <=> cast(0 as double) FROM t -- !query schema -struct<(false <=> CAST(0 AS DOUBLE)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as double)" + } ] +} -- !query SELECT false <=> cast(0 as decimal(10, 0)) FROM t -- !query schema -struct<(false <=> CAST(0 AS DECIMAL(10,0))):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "false <=> cast(0 as decimal(10, 0))" + } ] +} -- !query @@ -874,57 +1546,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(0 as tinyint) = false FROM t -- !query schema -struct<(CAST(0 AS TINYINT) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as tinyint) = false" + } ] +} -- !query SELECT cast(0 as smallint) = false FROM t -- !query schema -struct<(CAST(0 AS SMALLINT) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as smallint) = false" + } ] +} -- !query SELECT cast(0 as int) = false FROM t -- !query schema -struct<(CAST(0 AS INT) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast(0 as int) = false" + } ] +} -- !query SELECT cast(0 as bigint) = false FROM t -- !query schema -struct<(CAST(0 AS BIGINT) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(0 as bigint) = false" + } ] +} -- !query SELECT cast(0 as float) = false FROM t -- !query schema -struct<(CAST(0 AS FLOAT) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(0 as float) = false" + } ] +} -- !query SELECT cast(0 as double) = false FROM t -- !query schema -struct<(CAST(0 AS DOUBLE) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(0 as double) = false" + } ] +} -- !query SELECT cast(0 as decimal(10, 0)) = false FROM t -- !query schema -struct<(CAST(0 AS DECIMAL(10,0)) = false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(0 as decimal(10, 0)) = false" + } ] +} -- !query @@ -1018,57 +1802,169 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(0 as tinyint) <=> false FROM t -- !query schema -struct<(CAST(0 AS TINYINT) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(0 as tinyint) <=> false" + } ] +} -- !query SELECT cast(0 as smallint) <=> false FROM t -- !query schema -struct<(CAST(0 AS SMALLINT) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast(0 as smallint) <=> false" + } ] +} -- !query SELECT cast(0 as int) <=> false FROM t -- !query schema -struct<(CAST(0 AS INT) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(0 as int) <=> false" + } ] +} -- !query SELECT cast(0 as bigint) <=> false FROM t -- !query schema -struct<(CAST(0 AS BIGINT) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as bigint) <=> false" + } ] +} -- !query SELECT cast(0 as float) <=> false FROM t -- !query schema -struct<(CAST(0 AS FLOAT) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as float) <=> false" + } ] +} -- !query SELECT cast(0 as double) <=> false FROM t -- !query schema -struct<(CAST(0 AS DOUBLE) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as double) <=> false" + } ] +} -- !query SELECT cast(0 as decimal(10, 0)) <=> false FROM t -- !query schema -struct<(CAST(0 AS DECIMAL(10,0)) <=> false):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(0 as decimal(10, 0)) <=> false" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out index 7973d11573767..35ff9e79d9808 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -42,7 +42,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -66,7 +66,7 @@ struct +struct -- !query output 1 @@ -202,7 +202,7 @@ struct +struct -- !query output 1.0 @@ -226,7 +226,7 @@ struct +struct -- !query output 1 @@ -362,7 +362,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as float) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -386,7 +386,7 @@ struct +struct -- !query output 1 @@ -522,7 +522,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as float) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -546,7 +546,7 @@ struct +struct -- !query output 1 @@ -650,7 +650,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as tinyint) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -658,7 +658,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as smallint) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -666,7 +666,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as int) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -674,7 +674,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as bigint) END FROM t -- !query schema -struct +struct -- !query output 1.0 @@ -706,7 +706,7 @@ struct +struct -- !query output 1.0 @@ -866,7 +866,7 @@ struct +struct -- !query output 1.0 @@ -1026,9 +1026,9 @@ struct +struct -- !query output -1 +1.0 -- !query @@ -1130,7 +1130,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as tinyint) END FROM t -- !query schema -struct +struct -- !query output 1 @@ -1138,7 +1138,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as smallint) END FROM t -- !query schema -struct +struct -- !query output 1 @@ -1146,7 +1146,7 @@ struct +struct -- !query output 1 @@ -1154,7 +1154,7 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as bigint) END FROM t -- !query schema -struct +struct -- !query output 1 @@ -1162,25 +1162,25 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as float) END FROM t -- !query schema -struct +struct -- !query output -1 +1.0 -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as double) END FROM t -- !query schema -struct +struct -- !query output -1 +1.0 -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as decimal(10, 0)) END FROM t -- !query schema -struct +struct -- !query output -1 +1.0 -- !query @@ -1194,67 +1194,67 @@ struct -- !query SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t -- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BINARY) END\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 73, - "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END" + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" } ] } -- !query -SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 72, - "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END" + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END" } ] } --- !query -SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t --- !query schema -struct --- !query output -1 - - --- !query -SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t --- !query schema -struct --- !query output -1 - - -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM t -- !query schema @@ -1426,25 +1426,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS STRING) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 73, - "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END" - } ] -} +1 -- !query @@ -1698,25 +1682,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`casewhen`", - "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 72, - "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END" - } ] -} +true -- !query @@ -1970,7 +1938,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as string) END FROM t -- !query schema -struct +struct -- !query output 2017-12-12 09:30:00 @@ -2210,7 +2178,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as string) END FROM t -- !query schema -struct +struct -- !query output 2017-12-12 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out index bb02058f6c4ad..0f42834d28246 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/concat.sql.out @@ -311,6 +311,22 @@ SELECT (string_array1 || int_array2) sti_array FROM various_arrays -- !query schema -struct,si_array:array,ib_array:array,bd_array:array,dd_array:array,df_array:array,std_array:array,tst_array:array,sti_array:array> +struct<> -- !query output -[2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,9223372036854775808,9223372036854775809] [9.223372036854776E18,9.223372036854776E18,3.0,4.0] [2.0,1.0,3.0,4.0] ["a","b","2016-03-12","2016-03-11"] ["2016-11-15 20:54:00","2016-11-12 20:54:00","c","d"] ["a","b","3","4"] +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 295, + "stopIndex" : 322, + "fragment" : "string_array1 || data_array2" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out index b23e57c470fe4..54e26851ba57e 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -7834,33 +7834,97 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) = cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) = cast(1 as boolean)" + } ] +} -- !query @@ -8922,33 +8986,97 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t -- !query schema -struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BOOLEAN)):boolean> +struct<> -- !query output -true +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast(1 as boolean)" + } ] +} -- !query @@ -14618,33 +14746,97 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t -- !query schema -struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t -- !query schema -struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t -- !query schema -struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) <> cast(1 as boolean)" + } ] +} -- !query SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t -- !query schema -struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))):boolean> +struct<> -- !query output -false +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) <> cast(1 as boolean)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out index 5f16135fcaf4f..cf6931a4ffdaa 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/division.sql.out @@ -1186,9 +1186,25 @@ struct<(CAST(1 AS STRING) / CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT cast(1 as string) / cast(1 as string) FROM t -- !query schema -struct<(CAST(1 AS STRING) / CAST(1 AS STRING)):double> +struct<> -- !query output -1.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as string) / cast(1 as string)" + } ] +} -- !query @@ -1198,11 +1214,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -1222,11 +1238,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -1246,11 +1262,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -1270,11 +1286,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -1462,11 +1478,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -1750,11 +1766,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -2038,11 +2054,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"" }, "queryContext" : [ { @@ -2326,11 +2342,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"" }, "queryContext" : [ { diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out index c0c3cefab8413..b2e0f50028a4e 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/ifCoercion.sql.out @@ -42,7 +42,7 @@ struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS BIGINT))):bigint> -- !query SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS FLOAT))):float> +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS FLOAT))):double> -- !query output 1.0 @@ -66,7 +66,7 @@ struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> -- !query SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS STRING))):bigint> -- !query output 1 @@ -202,7 +202,7 @@ struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BIGINT))):bigint> -- !query SELECT IF(true, cast(1 as smallint), cast(2 as float)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS FLOAT))):float> +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS FLOAT))):double> -- !query output 1.0 @@ -226,7 +226,7 @@ struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> -- !query SELECT IF(true, cast(1 as smallint), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS STRING))):bigint> -- !query output 1 @@ -362,7 +362,7 @@ struct<(IF(true, CAST(1 AS INT), CAST(2 AS BIGINT))):bigint> -- !query SELECT IF(true, cast(1 as int), cast(2 as float)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS INT), CAST(2 AS FLOAT))):float> +struct<(IF(true, CAST(1 AS INT), CAST(2 AS FLOAT))):double> -- !query output 1.0 @@ -386,7 +386,7 @@ struct<(IF(true, CAST(1 AS INT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> -- !query SELECT IF(true, cast(1 as int), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS INT), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS INT), CAST(2 AS STRING))):bigint> -- !query output 1 @@ -522,7 +522,7 @@ struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS BIGINT))):bigint> -- !query SELECT IF(true, cast(1 as bigint), cast(2 as float)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS FLOAT))):float> +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS FLOAT))):double> -- !query output 1.0 @@ -546,7 +546,7 @@ struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS DECIMAL(10,0)))):decimal(20,0)> -- !query SELECT IF(true, cast(1 as bigint), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS STRING))):bigint> -- !query output 1 @@ -650,7 +650,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as float), cast(2 as tinyint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT))):float> +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT))):double> -- !query output 1.0 @@ -658,7 +658,7 @@ struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT))):float> -- !query SELECT IF(true, cast(1 as float), cast(2 as smallint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT))):float> +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT))):double> -- !query output 1.0 @@ -666,7 +666,7 @@ struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT))):float> -- !query SELECT IF(true, cast(1 as float), cast(2 as int)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS INT))):float> +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS INT))):double> -- !query output 1.0 @@ -674,7 +674,7 @@ struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS INT))):float> -- !query SELECT IF(true, cast(1 as float), cast(2 as bigint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS BIGINT))):float> +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS BIGINT))):double> -- !query output 1.0 @@ -706,7 +706,7 @@ struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS DECIMAL(10,0)))):double> -- !query SELECT IF(true, cast(1 as float), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS STRING))):double> -- !query output 1.0 @@ -866,7 +866,7 @@ struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS DECIMAL(10,0)))):double> -- !query SELECT IF(true, cast(1 as double), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS STRING))):double> -- !query output 1.0 @@ -1026,9 +1026,9 @@ struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10 -- !query SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS STRING))):double> -- !query output -1 +1.0 -- !query @@ -1130,7 +1130,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as string), cast(2 as tinyint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT))):bigint> -- !query output 1 @@ -1138,7 +1138,7 @@ struct<(IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT))):string> -- !query SELECT IF(true, cast(1 as string), cast(2 as smallint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT))):bigint> -- !query output 1 @@ -1146,7 +1146,7 @@ struct<(IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT))):string> -- !query SELECT IF(true, cast(1 as string), cast(2 as int)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS INT))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS INT))):bigint> -- !query output 1 @@ -1154,7 +1154,7 @@ struct<(IF(true, CAST(1 AS STRING), CAST(2 AS INT))):string> -- !query SELECT IF(true, cast(1 as string), cast(2 as bigint)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT))):bigint> -- !query output 1 @@ -1162,25 +1162,25 @@ struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT))):string> -- !query SELECT IF(true, cast(1 as string), cast(2 as float)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS FLOAT))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS FLOAT))):double> -- !query output -1 +1.0 -- !query SELECT IF(true, cast(1 as string), cast(2 as double)) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DOUBLE))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DOUBLE))):double> -- !query output -1 +1.0 -- !query SELECT IF(true, cast(1 as string), cast(2 as decimal(10, 0))) FROM t -- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DECIMAL(10,0)))):string> +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DECIMAL(10,0)))):double> -- !query output -1 +1.0 -- !query @@ -1194,67 +1194,67 @@ struct<(IF(true, CAST(1 AS STRING), CAST(2 AS STRING))):string> -- !query SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t -- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BINARY))):binary> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BINARY)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "IF(true, cast(1 as string), cast('2' as binary))" + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp))" } ] } -- !query -SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "IF(true, cast(1 as string), cast(2 as boolean))" + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date))" } ] } --- !query -SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):string> --- !query output -1 - - --- !query -SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t --- !query schema -struct<(IF(true, CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE))):string> --- !query output -1 - - -- !query SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t -- !query schema @@ -1426,25 +1426,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t -- !query schema -struct<> +struct<(IF(true, CAST(1 AS BINARY), CAST(2 AS STRING))):binary> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 55, - "fragment" : "IF(true, cast('1' as binary), cast(2 as string))" - } ] -} +1 -- !query @@ -1698,25 +1682,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t -- !query schema -struct<> +struct<(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING))):boolean> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`if`", - "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 54, - "fragment" : "IF(true, cast(1 as boolean), cast(2 as string))" - } ] -} +true -- !query @@ -1970,7 +1938,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS STRING))):timestamp> -- !query output 2017-12-12 09:30:00 @@ -2210,7 +2178,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as string)) FROM t -- !query schema -struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS STRING))):string> +struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS STRING))):date> -- !query output 2017-12-12 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out index a746500c746f9..bb75fe5991acf 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out @@ -10,25 +10,25 @@ struct<> -- !query SELECT 1 + '2' FROM t -- !query schema -struct<(1 + 2):double> +struct<(1 + 2):bigint> -- !query output -3.0 +3 -- !query SELECT 1 - '2' FROM t -- !query schema -struct<(1 - 2):double> +struct<(1 - 2):bigint> -- !query output --1.0 +-1 -- !query SELECT 1 * '2' FROM t -- !query schema -struct<(1 * 2):double> +struct<(1 * 2):bigint> -- !query output -2.0 +2 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out index 3ca78fa17a64b..7c9152a66a9c1 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/inConversion.sql.out @@ -708,7 +708,7 @@ SELECT cast(1 as float) in (cast(1 as string)) FROM t -- !query schema struct<(CAST(1 AS FLOAT) IN (CAST(1 AS STRING))):boolean> -- !query output -false +true -- !query @@ -868,7 +868,7 @@ SELECT cast(1 as double) in (cast(1 as string)) FROM t -- !query schema struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS STRING))):boolean> -- !query output -false +true -- !query @@ -1164,7 +1164,7 @@ SELECT cast(1 as string) in (cast(1 as float)) FROM t -- !query schema struct<(CAST(1 AS STRING) IN (CAST(1 AS FLOAT))):boolean> -- !query output -false +true -- !query @@ -1172,7 +1172,7 @@ SELECT cast(1 as string) in (cast(1 as double)) FROM t -- !query schema struct<(CAST(1 AS STRING) IN (CAST(1 AS DOUBLE))):boolean> -- !query output -false +true -- !query @@ -1194,67 +1194,67 @@ true -- !query SELECT cast(1 as string) in (cast('1' as binary)) FROM t -- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as boolean)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BINARY\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BINARY)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 26, - "stopIndex" : 49, - "fragment" : "in (cast('1' as binary))" + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" } ] } -- !query -SELECT cast(1 as string) in (cast(1 as boolean)) FROM t +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"BOOLEAN\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 26, - "stopIndex" : 48, - "fragment" : "in (cast(1 as boolean))" + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" } ] } --- !query -SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query schema -struct<(CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> --- !query output -false - - --- !query -SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t --- !query schema -struct<(CAST(1 AS STRING) IN (CAST(2017-12-11 09:30:00 AS DATE))):boolean> --- !query output -false - - -- !query SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t -- !query schema @@ -1426,25 +1426,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) in (cast(1 as string)) FROM t -- !query schema -struct<> +struct<(CAST(1 AS BINARY) IN (CAST(1 AS STRING))):boolean> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 49, - "fragment" : "in (cast(1 as string))" - } ] -} +true -- !query @@ -1698,25 +1682,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT true in (cast(1 as string)) FROM t -- !query schema -struct<> +struct<(true IN (CAST(1 AS STRING))):boolean> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(true IN (CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 13, - "stopIndex" : 34, - "fragment" : "in (cast(1 as string))" - } ] -} +true -- !query @@ -1970,9 +1938,25 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as string)) FROM t -- !query schema -struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS STRING))):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as string))" + } ] +} -- !query @@ -2210,9 +2194,25 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as string)) FROM t -- !query schema -struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS STRING))):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as string))" + } ] +} -- !query @@ -3466,67 +3466,67 @@ true -- !query SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t -- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"STRING\", \"BINARY\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 26, - "stopIndex" : 68, - "fragment" : "in (cast(1 as string), cast('1' as binary))" + "stopIndex" : 91, + "fragment" : "in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp))" } ] } -- !query -SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "dataType" : "[\"STRING\", \"STRING\", \"BOOLEAN\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN)))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 26, - "stopIndex" : 67, - "fragment" : "in (cast(1 as string), cast(1 as boolean))" + "stopIndex" : 84, + "fragment" : "in (cast(1 as string), cast('2017-12-11 09:30:00' as date))" } ] } --- !query -SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t --- !query schema -struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> --- !query output -true - - --- !query -SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t --- !query schema -struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(2017-12-11 09:30:00 AS DATE))):boolean> --- !query output -true - - -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t -- !query schema @@ -3698,25 +3698,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t -- !query schema -struct<> +struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING))):boolean> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BINARY\", \"BINARY\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 28, - "stopIndex" : 70, - "fragment" : "in (cast('1' as binary), cast(1 as string))" - } ] -} +true -- !query @@ -3970,25 +3954,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t -- !query schema -struct<> +struct<(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING))):boolean> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", - "sqlState" : "42K09", - "messageParameters" : { - "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"STRING\"]", - "functionName" : "`in`", - "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING)))\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 29, - "stopIndex" : 72, - "fragment" : "in (cast('1' as boolean), cast(1 as string))" - } ] -} +true -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index 5c00e8a5b63db..a0bd111f6ba64 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -124,36 +124,104 @@ struct>> SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query schema -struct>> +struct<> -- !query output -{"2":{"k":"2","v1":"1","v2":1},"true":{"k":"true","v1":"false","v2":null}} +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_zip_with(string_map1, int_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query schema -struct>> +struct<> -- !query output -{"2016-03-14":{"k":"2016-03-14","v1":"2016-03-13","v2":2016-03-13}} +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"DATE\"", + "sqlExpr" : "\"map_zip_with(string_map2, date_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query schema -struct>> +struct<> -- !query output -{"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 20:54:00,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}} +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"TIMESTAMP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(timestamp_map, string_map3, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m FROM various_maps -- !query schema -struct>> +struct<> -- !query output -{"922337203685477897945456575809789456":{"k":"922337203685477897945456575809789456","v1":922337203685477897945456575809789456,"v2":"text"}} +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"DECIMAL(36,0)\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(decimal_map1, string_map4, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index 8fd398ff87f0b..893e9b511986b 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -78,9 +78,25 @@ SELECT map_concat(int_string_map1, tinyint_map2) istt_map FROM various_maps -- !query schema -struct,si_map:map,ib_map:map,bd_map:map,df_map:map,std_map:map,tst_map:map,sti_map:map,istt_map:map> +struct<> -- !query output -{1:2,3:4} {1:2,7:8} {4:6,8:9} {6:7,9223372036854775808:9223372036854775809} {3.0:4.0,9.223372036854776E18:9.223372036854776E18} {"2016-03-12":"2016-03-11","a":"b"} {"2016-11-15 20:54:00":"2016-11-12 20:54:00","c":"d"} {"7":"8","a":"b"} {1:"a",3:"4"} +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(string_map1, date_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 257, + "stopIndex" : 290, + "fragment" : "map_concat(string_map1, date_map2)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out index 5005d682e1927..a97abd5dd181b 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -10,33 +10,33 @@ struct<> -- !query SELECT '1' + cast(1 as tinyint) FROM t -- !query schema -struct<(1 + CAST(1 AS TINYINT)):double> +struct<(1 + CAST(1 AS TINYINT)):bigint> -- !query output -2.0 +2 -- !query SELECT '1' + cast(1 as smallint) FROM t -- !query schema -struct<(1 + CAST(1 AS SMALLINT)):double> +struct<(1 + CAST(1 AS SMALLINT)):bigint> -- !query output -2.0 +2 -- !query SELECT '1' + cast(1 as int) FROM t -- !query schema -struct<(1 + CAST(1 AS INT)):double> +struct<(1 + CAST(1 AS INT)):bigint> -- !query output -2.0 +2 -- !query SELECT '1' + cast(1 as bigint) FROM t -- !query schema -struct<(1 + CAST(1 AS BIGINT)):double> +struct<(1 + CAST(1 AS BIGINT)):bigint> -- !query output -2.0 +2 -- !query @@ -66,9 +66,25 @@ struct<(1 + CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT '1' + '1' FROM t -- !query schema -struct<(1 + 1):double> +struct<> -- !query output -2.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' + '1'" + } ] +} -- !query @@ -78,11 +94,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -102,11 +118,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -126,11 +142,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -153,11 +169,11 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", "sqlState" : "42K09", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputSql" : "\"CAST(2017-12-11 09:30:00 AS DATE)\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + "sqlExpr" : "\"date_add(1, CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { "objectType" : "", @@ -172,33 +188,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' - cast(1 as tinyint) FROM t -- !query schema -struct<(1 - CAST(1 AS TINYINT)):double> +struct<(1 - CAST(1 AS TINYINT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' - cast(1 as smallint) FROM t -- !query schema -struct<(1 - CAST(1 AS SMALLINT)):double> +struct<(1 - CAST(1 AS SMALLINT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' - cast(1 as int) FROM t -- !query schema -struct<(1 - CAST(1 AS INT)):double> +struct<(1 - CAST(1 AS INT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' - cast(1 as bigint) FROM t -- !query schema -struct<(1 - CAST(1 AS BIGINT)):double> +struct<(1 - CAST(1 AS BIGINT)):bigint> -- !query output -0.0 +0 -- !query @@ -228,9 +244,25 @@ struct<(1 - CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT '1' - '1' FROM t -- !query schema -struct<(1 - 1):double> +struct<> -- !query output -0.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' - '1'" + } ] +} -- !query @@ -240,11 +272,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -264,11 +296,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -286,16 +318,14 @@ SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(1 - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -310,41 +340,57 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 - CAST(2017-12-11 09:30:00 AS DATE)):interval day> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' - cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query SELECT '1' * cast(1 as tinyint) FROM t -- !query schema -struct<(1 * CAST(1 AS TINYINT)):double> +struct<(1 * CAST(1 AS TINYINT)):bigint> -- !query output -1.0 +1 -- !query SELECT '1' * cast(1 as smallint) FROM t -- !query schema -struct<(1 * CAST(1 AS SMALLINT)):double> +struct<(1 * CAST(1 AS SMALLINT)):bigint> -- !query output -1.0 +1 -- !query SELECT '1' * cast(1 as int) FROM t -- !query schema -struct<(1 * CAST(1 AS INT)):double> +struct<(1 * CAST(1 AS INT)):bigint> -- !query output -1.0 +1 -- !query SELECT '1' * cast(1 as bigint) FROM t -- !query schema -struct<(1 * CAST(1 AS BIGINT)):double> +struct<(1 * CAST(1 AS BIGINT)):bigint> -- !query output -1.0 +1 -- !query @@ -374,9 +420,25 @@ struct<(1 * CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT '1' * '1' FROM t -- !query schema -struct<(1 * 1):double> +struct<> -- !query output -1.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' * '1'" + } ] +} -- !query @@ -386,11 +448,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -410,11 +472,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -434,11 +496,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -458,11 +520,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -534,9 +596,25 @@ struct<(1 / CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT '1' / '1' FROM t -- !query schema -struct<(1 / 1):double> +struct<> -- !query output -1.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' / '1'" + } ] +} -- !query @@ -546,11 +624,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -570,11 +648,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -594,11 +672,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -618,11 +696,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -638,33 +716,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT '1' % cast(1 as tinyint) FROM t -- !query schema -struct<(1 % CAST(1 AS TINYINT)):double> +struct<(1 % CAST(1 AS TINYINT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' % cast(1 as smallint) FROM t -- !query schema -struct<(1 % CAST(1 AS SMALLINT)):double> +struct<(1 % CAST(1 AS SMALLINT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' % cast(1 as int) FROM t -- !query schema -struct<(1 % CAST(1 AS INT)):double> +struct<(1 % CAST(1 AS INT)):bigint> -- !query output -0.0 +0 -- !query SELECT '1' % cast(1 as bigint) FROM t -- !query schema -struct<(1 % CAST(1 AS BIGINT)):double> +struct<(1 % CAST(1 AS BIGINT)):bigint> -- !query output -0.0 +0 -- !query @@ -694,9 +772,25 @@ struct<(1 % CAST(1 AS DECIMAL(10,0))):double> -- !query SELECT '1' % '1' FROM t -- !query schema -struct<(1 % 1):double> +struct<> -- !query output -0.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' % '1'" + } ] +} -- !query @@ -706,11 +800,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -730,11 +824,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -754,11 +848,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -778,11 +872,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -798,33 +892,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT pmod('1', cast(1 as tinyint)) FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod('1', cast(1 as smallint)) FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod('1', cast(1 as int)) FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod('1', cast(1 as bigint)) FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query @@ -854,9 +948,25 @@ struct -- !query SELECT pmod('1', '1') FROM t -- !query schema -struct +struct<> -- !query output -0.0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "pmod('1', '1')" + } ] +} -- !query @@ -866,11 +976,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BINARY\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"" }, "queryContext" : [ { @@ -890,11 +1000,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"BOOLEAN\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"" }, "queryContext" : [ { @@ -914,11 +1024,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"TIMESTAMP\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" }, "queryContext" : [ { @@ -938,11 +1048,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DOUBLE\"", - "right" : "\"DATE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"" }, "queryContext" : [ { @@ -958,33 +1068,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) + '1' FROM t -- !query schema -struct<(CAST(1 AS TINYINT) + 1):double> +struct<(CAST(1 AS TINYINT) + 1):bigint> -- !query output -2.0 +2 -- !query SELECT cast(1 as smallint) + '1' FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) + 1):double> +struct<(CAST(1 AS SMALLINT) + 1):bigint> -- !query output -2.0 +2 -- !query SELECT cast(1 as int) + '1' FROM t -- !query schema -struct<(CAST(1 AS INT) + 1):double> +struct<(CAST(1 AS INT) + 1):bigint> -- !query output -2.0 +2 -- !query SELECT cast(1 as bigint) + '1' FROM t -- !query schema -struct<(CAST(1 AS BIGINT) + 1):double> +struct<(CAST(1 AS BIGINT) + 1):bigint> -- !query output -2.0 +2 -- !query @@ -1018,11 +1128,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"" }, "queryContext" : [ { @@ -1042,11 +1152,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"" }, "queryContext" : [ { @@ -1066,11 +1176,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"" }, "queryContext" : [ { @@ -1094,7 +1204,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", + "inputType" : "\"DATE\"", "paramIndex" : "second", "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" @@ -1112,33 +1222,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) - '1' FROM t -- !query schema -struct<(CAST(1 AS TINYINT) - 1):double> +struct<(CAST(1 AS TINYINT) - 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as smallint) - '1' FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) - 1):double> +struct<(CAST(1 AS SMALLINT) - 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as int) - '1' FROM t -- !query schema -struct<(CAST(1 AS INT) - 1):double> +struct<(CAST(1 AS INT) - 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as bigint) - '1' FROM t -- !query schema -struct<(CAST(1 AS BIGINT) - 1):double> +struct<(CAST(1 AS BIGINT) - 1):bigint> -- !query output -0.0 +0 -- !query @@ -1172,11 +1282,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"" }, "queryContext" : [ { @@ -1196,11 +1306,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"" }, "queryContext" : [ { @@ -1218,16 +1328,14 @@ SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"STRING\"", - "paramIndex" : "second", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - 1)\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", @@ -1244,16 +1352,14 @@ SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "inputSql" : "\"1\"", - "inputType" : "\"DOUBLE\"", - "paramIndex" : "second", - "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", - "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", @@ -1268,33 +1374,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) * '1' FROM t -- !query schema -struct<(CAST(1 AS TINYINT) * 1):double> +struct<(CAST(1 AS TINYINT) * 1):bigint> -- !query output -1.0 +1 -- !query SELECT cast(1 as smallint) * '1' FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) * 1):double> +struct<(CAST(1 AS SMALLINT) * 1):bigint> -- !query output -1.0 +1 -- !query SELECT cast(1 as int) * '1' FROM t -- !query schema -struct<(CAST(1 AS INT) * 1):double> +struct<(CAST(1 AS INT) * 1):bigint> -- !query output -1.0 +1 -- !query SELECT cast(1 as bigint) * '1' FROM t -- !query schema -struct<(CAST(1 AS BIGINT) * 1):double> +struct<(CAST(1 AS BIGINT) * 1):bigint> -- !query output -1.0 +1 -- !query @@ -1328,11 +1434,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"" }, "queryContext" : [ { @@ -1352,11 +1458,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"" }, "queryContext" : [ { @@ -1376,11 +1482,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"" }, "queryContext" : [ { @@ -1400,11 +1506,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"" }, "queryContext" : [ { @@ -1480,11 +1586,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"" }, "queryContext" : [ { @@ -1504,11 +1610,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"" }, "queryContext" : [ { @@ -1528,11 +1634,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"" }, "queryContext" : [ { @@ -1552,11 +1658,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"" }, "queryContext" : [ { @@ -1572,33 +1678,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as tinyint) % '1' FROM t -- !query schema -struct<(CAST(1 AS TINYINT) % 1):double> +struct<(CAST(1 AS TINYINT) % 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as smallint) % '1' FROM t -- !query schema -struct<(CAST(1 AS SMALLINT) % 1):double> +struct<(CAST(1 AS SMALLINT) % 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as int) % '1' FROM t -- !query schema -struct<(CAST(1 AS INT) % 1):double> +struct<(CAST(1 AS INT) % 1):bigint> -- !query output -0.0 +0 -- !query SELECT cast(1 as bigint) % '1' FROM t -- !query schema -struct<(CAST(1 AS BIGINT) % 1):double> +struct<(CAST(1 AS BIGINT) % 1):bigint> -- !query output -0.0 +0 -- !query @@ -1632,11 +1738,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"" }, "queryContext" : [ { @@ -1656,11 +1762,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"" }, "queryContext" : [ { @@ -1680,11 +1786,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"" }, "queryContext" : [ { @@ -1704,11 +1810,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"" }, "queryContext" : [ { @@ -1724,33 +1830,33 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT pmod(cast(1 as tinyint), '1') FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod(cast(1 as smallint), '1') FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod(cast(1 as int), '1') FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query SELECT pmod(cast(1 as bigint), '1') FROM t -- !query schema -struct +struct -- !query output -0.0 +0 -- !query @@ -1784,11 +1890,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BINARY\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"" }, "queryContext" : [ { @@ -1808,11 +1914,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"BOOLEAN\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"" }, "queryContext" : [ { @@ -1832,11 +1938,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"TIMESTAMP\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"" }, "queryContext" : [ { @@ -1856,11 +1962,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", "sqlState" : "42K09", "messageParameters" : { - "left" : "\"DATE\"", - "right" : "\"DOUBLE\"", + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" }, "queryContext" : [ { @@ -1956,17 +2062,49 @@ true -- !query SELECT '1' = cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 = CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' = cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2044,17 +2182,49 @@ true -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) = '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) = '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) = 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) = '1'" + } ] +} -- !query @@ -2140,17 +2310,49 @@ true -- !query SELECT '1' <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "'1' <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 <=> CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "'1' <=> cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2228,17 +2430,49 @@ true -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> 1):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) <=> '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) <=> 1):boolean> +struct<> -- !query output -false +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> '1'" + } ] +} -- !query @@ -2324,17 +2558,49 @@ false -- !query SELECT '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 < CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' < cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2420,17 +2686,49 @@ true -- !query SELECT '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 <= CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' <= cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2516,17 +2814,49 @@ false -- !query SELECT '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 > CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' > cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2612,17 +2942,49 @@ true -- !query SELECT '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(1 >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(1 >= CAST(2017-12-11 09:30:00 AS DATE)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' >= cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2708,17 +3070,49 @@ false -- !query SELECT '1' <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query schema -struct<(NOT (1 = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} -- !query SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t -- !query schema -struct<(NOT (1 = CAST(2017-12-11 09:30:00 AS DATE))):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' <> cast('2017-12-11 09:30:00' as date)" + } ] +} -- !query @@ -2804,17 +3198,49 @@ false -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) < '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) < '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) < 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) < '1'" + } ] +} -- !query @@ -2900,17 +3326,49 @@ true -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) <= '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) <= 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= '1'" + } ] +} -- !query @@ -2996,17 +3454,49 @@ false -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) > '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) > '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) > 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) > '1'" + } ] +} -- !query @@ -3092,17 +3582,49 @@ true -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) >= '1' FROM t -- !query schema -struct<(CAST(2017-12-11 09:30:00 AS DATE) >= 1):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= '1'" + } ] +} -- !query @@ -3188,17 +3710,49 @@ false -- !query SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> '1' FROM t -- !query schema -struct<(NOT (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = 1)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> '1'" + } ] +} -- !query SELECT cast('2017-12-11 09:30:00' as date) <> '1' FROM t -- !query schema -struct<(NOT (CAST(2017-12-11 09:30:00 AS DATE) = 1)):boolean> +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> '1'" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 08b70ff920eb6..781fff4835c51 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -10,81 +10,241 @@ struct<> -- !query select cast(a as byte) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TINYINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as byte)" + } ] +} -- !query select cast(a as short) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"SMALLINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "cast(a as short)" + } ] +} -- !query select cast(a as int) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "cast(a as int)" + } ] +} -- !query select cast(a as long) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as long)" + } ] +} -- !query select cast(a as float) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "cast(a as float)" + } ] +} -- !query select cast(a as double) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(a as double)" + } ] +} -- !query select cast(a as decimal) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast(a as decimal)" + } ] +} -- !query select cast(a as boolean) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast(a as boolean)" + } ] +} -- !query select cast(a as timestamp) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "cast(a as timestamp)" + } ] +} -- !query select cast(a as date) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as date)" + } ] +} -- !query @@ -170,9 +330,23 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query select to_timestamp(a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} -- !query @@ -195,9 +369,17 @@ org.apache.spark.SparkUpgradeException -- !query select to_unix_timestamp(a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'aa' could not be parsed at index 0" + } +} -- !query @@ -220,9 +402,17 @@ org.apache.spark.SparkUpgradeException -- !query select unix_timestamp(a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "message" : "Text 'aa' could not be parsed at index 0" + } +} -- !query @@ -245,41 +435,112 @@ org.apache.spark.SparkUpgradeException -- !query select from_unixtime(a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "from_unixtime(a)" + } ] +} -- !query select from_unixtime('2018-01-01', a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2018-01-01'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "from_unixtime('2018-01-01', a)" + } ] +} -- !query select next_day(a, 'MO') from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "next_day(a, 'MO')" + } ] +} -- !query select next_day('2018-01-01', a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "aa" + } +} -- !query select trunc(a, 'MM') from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "trunc(a, 'MM')" + } ] +} -- !query @@ -301,9 +562,25 @@ NULL -- !query select sha2(a, a) from t -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "sha2(a, a)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out index 79184489758bf..678d00a39b2d5 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -46,7 +46,7 @@ struct -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -73,7 +73,7 @@ struct -- !query SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -226,7 +226,7 @@ struct -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -253,7 +253,7 @@ struct -- !query SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -406,7 +406,7 @@ struct -- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -433,7 +433,7 @@ struct -- !query SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -586,7 +586,7 @@ struct -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -613,7 +613,7 @@ struct -- !query SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -730,7 +730,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -739,7 +739,7 @@ struct -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -748,7 +748,7 @@ struct -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -757,7 +757,7 @@ struct -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query schema -struct +struct -- !query output 1.0 2.0 @@ -793,10 +793,10 @@ struct -- !query SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1.0 -2 +2.0 -- !query @@ -973,10 +973,10 @@ struct -- !query SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output 1.0 -2 +2.0 -- !query @@ -1153,10 +1153,10 @@ struct -- !query SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct -- !query output -1 -2 +1.0 +2.0 -- !query @@ -1270,7 +1270,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -1279,7 +1279,7 @@ struct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -1288,7 +1288,7 @@ struct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -1297,7 +1297,7 @@ struct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t -- !query schema -struct +struct -- !query output 1 2 @@ -1306,28 +1306,28 @@ struct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t -- !query schema -struct +struct -- !query output -1 +1.0 2.0 -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t -- !query schema -struct +struct -- !query output -1 +1.0 2.0 -- !query SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t -- !query schema -struct +struct -- !query output -1 -2 +1.0 +2.0 -- !query @@ -1342,75 +1342,68 @@ struct -- !query SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t -- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"BINARY\"", - "dataType2" : "\"STRING\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t" + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" } ] } -- !query -SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkDateTimeException { - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"BOOLEAN\"", - "dataType2" : "\"STRING\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" }, "queryContext" : [ { "objectType" : "", "objectName" : "", "startIndex" : 1, - "stopIndex" : 70, - "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t" + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" } ] } --- !query -SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t --- !query schema -struct --- !query output -1 -2017-12-11 09:30:00 - - --- !query -SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t --- !query schema -struct --- !query output -1 -2017-12-11 - - -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t -- !query schema @@ -1603,28 +1596,10 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", - "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"STRING\"", - "dataType2" : "\"BINARY\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 1, - "stopIndex" : 71, - "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t" - } ] -} +1 +2 -- !query @@ -1911,17 +1886,14 @@ SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.SparkRuntimeException { - "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", - "sqlState" : "42825", + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", "messageParameters" : { - "columnOrdinalNumber" : "first", - "dataType1" : "\"STRING\"", - "dataType2" : "\"BOOLEAN\"", - "hint" : "", - "operator" : "UNION", - "tableOrdinalNumber" : "second" + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" }, "queryContext" : [ { "objectType" : "", @@ -2214,10 +2186,25 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct<> -- !query output -2 -2017-12-12 09:30:00 +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t" + } ] +} -- !query @@ -2484,10 +2471,25 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t -- !query schema -struct +struct<> -- !query output -2 -2017-12-12 +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out index f6d1120c75ff2..67645cfb732f5 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -191,13 +191,12 @@ struct<> -- !query output org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", "sqlState" : "42K09", "messageParameters" : { - "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", - "exprType" : "\"BINARY\"", - "location" : "upper", - "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + "orderSpecType" : "\"BINARY\"", + "sqlExpr" : "\"(PARTITION BY 1 ORDER BY CAST(1 AS BINARY) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" }, "queryContext" : [ { "objectType" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out index 1caeac58ab0ba..ad12f8bd03fd0 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out @@ -397,9 +397,25 @@ org.apache.spark.sql.AnalysisException -- !query SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) = false -- !query schema -struct +struct<> -- !query output -4 0 +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(udaf(v) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 65, + "fragment" : "udaf(v) = false" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out index ce70c91d8d06e..6a70c8b96841d 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -520,27 +520,9 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query SELECT udf(every("true")) -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"true\"", - "inputType" : "\"STRING\"", - "paramIndex" : "first", - "requiredType" : "\"BOOLEAN\"", - "sqlExpr" : "\"every(true)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 12, - "stopIndex" : 24, - "fragment" : "every(\"true\")" - } ] -} +true -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out index a355bdb16580a..7f2931c3ade0a 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-union.sql.out @@ -31,20 +31,31 @@ struct -- !query SELECT udf(c1) as c1, udf(c2) as c2 -FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a' UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 UNION ALL SELECT udf(c1) as c1, udf(c2) as c2 FROM t2) -- !query schema -struct +struct<> -- !query output -1.0 1 -1.0 1 -1.0 a -2.0 4 -2.0 4 -2.0 b +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 243, + "fragment" : "SELECT udf(c1) as c1, udf(c2) as c2\nFROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a'\n UNION ALL\n SELECT udf(c1) as c1, udf(c2) as c2 FROM t2\n UNION ALL\n SELECT udf(c1) as c1, udf(c2) as c2 FROM t2)" + } ] +} -- !query @@ -106,10 +117,25 @@ SELECT map(1, 2), udf('str') as str UNION ALL SELECT map(1, 2, 3, NULL), udf(1) -- !query schema -struct,str:string> +struct<> -- !query output -{1:2,3:null} 1 -{1:2} str +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT map(1, 2), udf('str') as str\nUNION ALL\nSELECT map(1, 2, 3, NULL), udf(1)" + } ] +} -- !query @@ -117,10 +143,25 @@ SELECT array(1, 2), udf('str') as str UNION ALL SELECT array(1, 2, 3, NULL), udf(1) -- !query schema -struct,str:string> +struct<> -- !query output -[1,2,3,null] 1 -[1,2] str +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT array(1, 2), udf('str') as str\nUNION ALL\nSELECT array(1, 2, 3, NULL), udf(1)" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index 451f514a21708..40e24e7b4e873 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -379,17 +379,23 @@ FROM testData WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) ORDER BY cate, udf(val) -- !query schema -struct,collect_set:array,skewness:double,kurtosis:double> +struct<> -- !query output -NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 -1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 -2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 -3 b 3 1 1 3 6 2.0 1.0 1 1 1 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1126, + "stopIndex" : 1161, + "fragment" : "corr(udf(val), udf(val_long)) OVER w" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index 674e58cd102f0..3825470777bdb 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -37,14 +37,25 @@ FROM (SELECT * FROM t1 UNION ALL SELECT * FROM t2) -- !query schema -struct +struct<> -- !query output -1.0 1 -1.0 1 -1.0 a -2.0 4 -2.0 4 -2.0 b +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 120, + "fragment" : "SELECT *\nFROM (SELECT * FROM t1\n UNION ALL\n SELECT * FROM t2\n UNION ALL\n SELECT * FROM t2)" + } ] +} -- !query @@ -106,10 +117,25 @@ SELECT map(1, 2), 'str' UNION ALL SELECT map(1, 2, 3, NULL), 1 -- !query schema -struct,str:string> +struct<> -- !query output -{1:2,3:null} 1 -{1:2} str +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT map(1, 2), 'str'\nUNION ALL\nSELECT map(1, 2, 3, NULL), 1" + } ] +} -- !query @@ -117,10 +143,25 @@ SELECT array(1, 2), 'str' UNION ALL SELECT array(1, 2, 3, NULL), 1 -- !query schema -struct,str:string> +struct<> -- !query output -[1,2,3,null] 1 -[1,2] str +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT array(1, 2), 'str'\nUNION ALL\nSELECT array(1, 2, 3, NULL), 1" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 96cf9e53787f7..182a4b819fcb0 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -557,17 +557,23 @@ FROM testData WINDOW w AS (PARTITION BY cate ORDER BY val) ORDER BY cate, val -- !query schema -struct,collect_set:array,skewness:double,kurtosis:double> +struct<> -- !query output -NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -3 NULL 3 3 3 1 3 3.0 NULL NULL 3 NULL NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NULL 1 0.0 NULL NULL 0.0 [3] [3] NULL NULL -NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 NULL NULL NULL NULL [] [] NULL NULL -1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 1 2 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 -1 a 1 1 1 2 2 1.0 0.0 NULL 1 NULL NULL 1 NULL 1 1 1 2 2 0.75 0.3333333333333333 2 3 0.0 0.0 1 0.0 NULL 0.0 0.0 [1,1] [1] 0.7071067811865476 -1.5 -2 a 2 1 1 3 4 1.3333333333333333 0.5773502691896258 NULL 1 NULL NULL 1 NULL 2 2 2 4 3 1.0 1.0 2 4 0.22222222222222224 0.33333333333333337 2 4.772185885555555E8 1.0 0.5773502691896258 0.4714045207910317 [1,1,2] [1,2] 1.1539890888012805 -0.6672217220327235 -1 b 1 1 1 1 1 1.0 NULL 1 1 1 1 1 1 1 1 1 1 1 0.3333333333333333 0.0 1 1 0.0 NULL 1 NULL NULL NULL 0.0 [1] [1] NULL NULL -2 b 2 1 1 2 3 1.5 0.7071067811865476 1 1 1 1 1 1 2 2 2 2 2 0.6666666666666666 0.5 1 2 0.25 0.5 2 0.0 NULL 0.7071067811865476 0.5 [1,2] [1,2] 0.0 -2.0000000000000013 -3 b 3 1 1 3 6 2.0 1.0 1 1 1 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 5.3687091175E8 1.0 1.0 0.816496580927726 [1,2,3] [1,2,3] 0.7057890433107311 -1.4999999999999984 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1016, + "stopIndex" : 1041, + "fragment" : "corr(val, val_long) OVER w" + } ] +} -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 4bd20bc245613..0872efd92002c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -185,8 +185,8 @@ trait SQLQueryTestHelper extends Logging { */ protected trait PgSQLTest - /** Trait that indicates ANSI-related tests with the ANSI mode enabled. */ - protected trait AnsiTest + /** Trait that indicates Non-ANSI-related tests with the ANSI mode disabled. */ + protected trait NonAnsiTest /** Trait that indicates an analyzer test that shows the analyzed plan string as output. */ protected trait AnalyzerTest extends TestCase { @@ -214,10 +214,10 @@ trait SQLQueryTestHelper extends Logging { } /** An ANSI-related test case. */ - protected case class AnsiTestCase( - name: String, inputFile: String, resultFile: String) extends TestCase with AnsiTest { + protected case class NonAnsiTestCase( + name: String, inputFile: String, resultFile: String) extends TestCase with NonAnsiTest { override def asAnalyzerTest(newName: String, newResultFile: String): TestCase = - AnsiAnalyzerTestCase(newName, inputFile, newResultFile) + NonAnsiAnalyzerTestCase(newName, inputFile, newResultFile) } /** An analyzer test that shows the analyzed plan string as output. */ @@ -290,9 +290,9 @@ trait SQLQueryTestHelper extends Logging { protected case class RegularAnalyzerTestCase( name: String, inputFile: String, resultFile: String) extends AnalyzerTest - protected case class AnsiAnalyzerTestCase( + protected case class NonAnsiAnalyzerTestCase( name: String, inputFile: String, resultFile: String) - extends AnalyzerTest with AnsiTest + extends AnalyzerTest with NonAnsiTest protected case class PgSQLAnalyzerTestCase( name: String, inputFile: String, resultFile: String) extends AnalyzerTest with PgSQLTest 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 b031f45ddbf34..5c56377f21c20 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 @@ -306,13 +306,13 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper localSparkSession.udf.register("vol", (s: String) => s) localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) localSparkSession.conf.set(SQLConf.LEGACY_INTERVAL_ENABLED.key, true) - case _: SQLQueryTestSuite#AnsiTest => - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + case _: SQLQueryTestSuite#NonAnsiTest => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, false) case _: SQLQueryTestSuite#TimestampNTZTest => localSparkSession.conf.set(SQLConf.TIMESTAMP_TYPE.key, TimestampTypes.TIMESTAMP_NTZ.toString) case _ => - localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, false) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) } if (sparkConfigSet.nonEmpty) { @@ -445,8 +445,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper } } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil - } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) { - AnsiTestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}nonansi")) { + NonAnsiTestCase(testCaseName, absPath, resultFile) :: Nil } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}timestampNTZ")) { TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}cte.sql")) { 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 782f549182ec2..662f43fc00399 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 @@ -124,13 +124,13 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ case _: SQLQueryTestSuite#PgSQLTest => statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") statement.execute(s"SET ${SQLConf.LEGACY_INTERVAL_ENABLED.key} = true") - case _: SQLQueryTestSuite#AnsiTest => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") + case _: SQLQueryTestSuite#NonAnsiTest => + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") case _: SQLQueryTestSuite#TimestampNTZTest => statement.execute(s"SET ${SQLConf.TIMESTAMP_TYPE.key} = " + s"${TimestampTypes.TIMESTAMP_NTZ.toString}") case _ => - statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = false") + statement.execute(s"SET ${SQLConf.ANSI_ENABLED.key} = true") } // Run the SQL queries preparing them for comparison. @@ -270,8 +270,8 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ Seq.empty } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil - } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) { - AnsiTestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}nonansi")) { + NonAnsiTestCase(testCaseName, absPath, resultFile) :: Nil } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}timestampNTZ")) { TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil } else { From a01856de20013e5551d385ee000772049a0e1bc0 Mon Sep 17 00:00:00 2001 From: Andrey Gubichev Date: Mon, 18 Nov 2024 15:39:27 +0800 Subject: [PATCH 02/58] [SPARK-50330][SQL] Add hints to Sort and Window nodes ### What changes were proposed in this pull request? Following https://github.com/apache/spark/pull/48523, this adds hints to Sort and Window nodes. ### Why are the changes needed? Allows the users to specify concrete hints. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test in SparkSessionExtensionSuite.scala ### Was this patch authored or co-authored using generative AI tooling? No Closes #48812 from agubichev/sort_window_hint. Authored-by: Andrey Gubichev Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 18 ++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 4 +- .../analysis/DeduplicateRelations.scala | 2 +- .../UnsupportedOperationChecker.scala | 4 +- .../optimizer/DecorrelateInnerQuery.scala | 10 ++--- .../optimizer/EliminateWindowPartitions.scala | 2 +- .../optimizer/InferWindowGroupLimit.scala | 2 +- .../LimitPushDownThroughWindow.scala | 4 +- .../optimizer/OptimizeOneRowPlan.scala | 4 +- .../sql/catalyst/optimizer/Optimizer.scala | 18 ++++----- .../optimizer/RemoveRedundantSorts.scala | 4 +- .../sql/catalyst/planning/patterns.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 6 ++- .../sql/catalyst/plans/logical/hints.scala | 4 ++ .../spark/sql/execution/SparkStrategies.scala | 14 +++---- .../v2/V2ScanRelationPushDown.scala | 3 +- .../apache/spark/sql/SQLQueryTestHelper.scala | 2 +- .../sql/SparkSessionExtensionSuite.scala | 37 ++++++++++++++++++- .../execution/python/PythonUDTFSuite.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 2 +- 20 files changed, 94 insertions(+), 52 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 23ecb54c095f5..bed7bea61597f 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 @@ -700,7 +700,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor f.copy(condition = newCond) // We should make sure all [[SortOrder]]s have been resolved. - case s @ Sort(order, _, child) + case s @ Sort(order, _, child, _) if order.exists(hasGroupingFunction) && order.forall(_.resolved) => val groupingExprs = findGroupingExprs(child) val gid = VirtualColumn.groupingIdAttribute @@ -1815,7 +1815,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case p if !p.childrenResolved => p // Replace the index with the related attribute for ORDER BY, // which is a 1-base position of the projection list. - case Sort(orders, global, child) + case Sort(orders, global, child, hint) if orders.exists(_.child.isInstanceOf[UnresolvedOrdinal]) => val newOrders = orders map { case s @ SortOrder(UnresolvedOrdinal(index), direction, nullOrdering, _) => @@ -1826,14 +1826,14 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } case o => o } - Sort(newOrders, global, child) + Sort(newOrders, global, child, hint) // Replace the index with the corresponding expression in aggregateExpressions. The index is // a 1-base position of aggregateExpressions, which is output columns (select expression) - case Aggregate(groups, aggs, child, _) if aggs.forall(_.resolved) && + case Aggregate(groups, aggs, child, hint) if aggs.forall(_.resolved) && groups.exists(containUnresolvedOrdinal) => val newGroups = groups.map(resolveGroupByExpressionOrdinal(_, aggs)) - Aggregate(newGroups, aggs, child) + Aggregate(newGroups, aggs, child, hint) } private def containUnresolvedOrdinal(e: Expression): Boolean = e match { @@ -2357,7 +2357,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor Filter(newExprs.head, newChild) }) - case s @ Sort(_, _, agg: Aggregate) if agg.resolved && s.order.forall(_.resolved) => + case s @ Sort(_, _, agg: Aggregate, _) if agg.resolved && s.order.forall(_.resolved) => resolveOperatorWithAggregate(s.order.map(_.child), agg, (newExprs, newChild) => { val newSortOrder = s.order.zip(newExprs).map { case (sortOrder, expr) => sortOrder.copy(child = expr) @@ -2365,7 +2365,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor s.copy(order = newSortOrder, child = newChild) }) - case s @ Sort(_, _, f @ Filter(cond, agg: Aggregate)) + case s @ Sort(_, _, f @ Filter(cond, agg: Aggregate), _) if agg.resolved && cond.resolved && s.order.forall(_.resolved) => resolveOperatorWithAggregate(s.order.map(_.child), agg, (newExprs, newChild) => { val newSortOrder = s.order.zip(newExprs).map { @@ -3618,10 +3618,10 @@ object CleanupAliases extends Rule[LogicalPlan] with AliasHelper { val cleanedAggs = aggs.map(trimNonTopLevelAliases) Aggregate(grouping.map(trimAliases), cleanedAggs, child, hint) - case Window(windowExprs, partitionSpec, orderSpec, child) => + case Window(windowExprs, partitionSpec, orderSpec, child, hint) => val cleanedWindowExprs = windowExprs.map(trimNonTopLevelAliases) Window(cleanedWindowExprs, partitionSpec.map(trimAliases), - orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child) + orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child, hint) case CollectMetrics(name, metrics, child, dataframeId) => val cleanedMetrics = metrics.map(trimNonTopLevelAliases) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index c7e5fa9f2b6c6..586a0312e1507 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -603,7 +603,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB case up: Unpivot if up.canBeCoercioned && !up.valuesTypeCoercioned => throw QueryCompilationErrors.unpivotValueDataTypeMismatchError(up.values.get) - case Sort(orders, _, _) => + case Sort(orders, _, _, _) => orders.foreach { order => if (!RowOrdering.isOrderable(order.dataType)) { order.failAnalysis( @@ -612,7 +612,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB } } - case Window(_, partitionSpec, _, _) => + case Window(_, partitionSpec, _, _, _) => // Both `partitionSpec` and `orderSpec` must be orderable. We only need an extra check // for `partitionSpec` here because `orderSpec` has the type check itself. partitionSpec.foreach { p => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index ca5a6eee9bc9d..c1535343d7686 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -423,7 +423,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { newVersion.copyTagsFrom(oldVersion) Seq((oldVersion, newVersion)) - case oldVersion @ Window(windowExpressions, _, _, child) + case oldVersion @ Window(windowExpressions, _, _, child, _) if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) .nonEmpty => val newVersion = oldVersion.copy(windowExpressions = newAliases(windowExpressions)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index 69639b69290c2..4f33c26d5c3c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -484,14 +484,14 @@ object UnsupportedOperationChecker extends Logging { case Offset(_, _) => throwError("Offset is not supported on streaming DataFrames/Datasets") - case Sort(_, _, _) if !containsCompleteData(subPlan) => + case Sort(_, _, _, _) if !containsCompleteData(subPlan) => throwError("Sorting is not supported on streaming DataFrames/Datasets, unless it is on " + "aggregated DataFrame/Dataset in Complete output mode") case Sample(_, _, _, _, child) if child.isStreaming => throwError("Sampling is not supported on streaming DataFrames/Datasets") - case Window(windowExpression, _, _, child) if child.isStreaming => + case Window(windowExpression, _, _, child, _) if child.isStreaming => val (windowFuncList, columnNameList, windowSpecList) = windowExpression.flatMap { e => e.collect { case we: WindowExpression => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala index 9758f37efc2dc..2b97b2621b5be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/DecorrelateInnerQuery.scala @@ -662,7 +662,7 @@ object DecorrelateInnerQuery extends PredicateHelper { // of limit in that case. This branch is for the case where there's no limit operator // above offset. val (child, ordering) = input match { - case Sort(order, _, child) => (child, order) + case Sort(order, _, child, _) => (child, order) case _ => (input, Seq()) } val (newChild, joinCond, outerReferenceMap) = @@ -705,8 +705,8 @@ object DecorrelateInnerQuery extends PredicateHelper { // SELECT T2.a, row_number() OVER (PARTITION BY T2.b ORDER BY T2.c) AS rn FROM T2) // WHERE rn > 2 AND rn <= 2+3 val (child, ordering, offsetExpr) = input match { - case Sort(order, _, child) => (child, order, Literal(0)) - case Offset(offsetExpr, offsetChild@(Sort(order, _, child))) => + case Sort(order, _, child, _) => (child, order, Literal(0)) + case Offset(offsetExpr, offsetChild@(Sort(order, _, child, _))) => (child, order, offsetExpr) case Offset(offsetExpr, child) => (child, Seq(), offsetExpr) @@ -754,7 +754,7 @@ object DecorrelateInnerQuery extends PredicateHelper { (project, joinCond, outerReferenceMap) } - case w @ Window(projectList, partitionSpec, orderSpec, child) => + case w @ Window(projectList, partitionSpec, orderSpec, child, hint) => val outerReferences = collectOuterReferences(w.expressions) assert(outerReferences.isEmpty, s"Correlated column is not allowed in window " + s"function: $w") @@ -770,7 +770,7 @@ object DecorrelateInnerQuery extends PredicateHelper { val newWindow = Window(newProjectList ++ referencesToAdd, partitionSpec = newPartitionSpec ++ referencesToAdd, - orderSpec = newOrderSpec, newChild) + orderSpec = newOrderSpec, newChild, hint) (newWindow, joinCond, outerReferenceMap) case a @ Aggregate(groupingExpressions, aggregateExpressions, child, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateWindowPartitions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateWindowPartitions.scala index e3d1b05443583..ca3ee12a3d7db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateWindowPartitions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateWindowPartitions.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.{WINDOW, WINDOW_EXPRESSIO object EliminateWindowPartitions extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( _.containsPattern(WINDOW), ruleId) { - case w @ Window(windowExprs, partitionSpec, _, _) if partitionSpec.exists(_.foldable) => + case w @ Window(windowExprs, partitionSpec, _, _, _) if partitionSpec.exists(_.foldable) => val newWindowExprs = windowExprs.map(_.transformWithPruning( _.containsPattern(WINDOW_EXPRESSION)) { case windowExpr @ WindowExpression(_, wsd @ WindowSpecDefinition(ps, _, _)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala index f2e99721e9261..46815969e7ece 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/InferWindowGroupLimit.scala @@ -74,7 +74,7 @@ object InferWindowGroupLimit extends Rule[LogicalPlan] with PredicateHelper { plan.transformWithPruning(_.containsAllPatterns(FILTER, WINDOW), ruleId) { case filter @ Filter(condition, - window @ Window(windowExpressions, partitionSpec, orderSpec, child)) + window @ Window(windowExpressions, partitionSpec, orderSpec, child, _)) if !child.isInstanceOf[WindowGroupLimit] && windowExpressions.forall(isExpandingWindow) && orderSpec.nonEmpty => val limits = windowExpressions.collect { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushDownThroughWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushDownThroughWindow.scala index de1368d28168f..c73d6ad8fa956 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushDownThroughWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushDownThroughWindow.scala @@ -43,14 +43,14 @@ object LimitPushDownThroughWindow extends Rule[LogicalPlan] { _.containsAllPatterns(WINDOW, LIMIT), ruleId) { // Adding an extra Limit below WINDOW when the partitionSpec of all window functions is empty. case LocalLimit(limitExpr @ IntegerLiteral(limit), - window @ Window(windowExpressions, Nil, orderSpec, child)) + window @ Window(windowExpressions, Nil, orderSpec, child, _)) if supportsPushdownThroughWindow(windowExpressions) && child.maxRows.forall(_ > limit) && limit < conf.topKSortFallbackThreshold => // Sort is needed here because we need global sort. window.copy(child = Limit(limitExpr, Sort(orderSpec, true, child))) // There is a Project between LocalLimit and Window if they do not have the same output. case LocalLimit(limitExpr @ IntegerLiteral(limit), project @ Project(_, - window @ Window(windowExpressions, Nil, orderSpec, child))) + window @ Window(windowExpressions, Nil, orderSpec, child, _))) if supportsPushdownThroughWindow(windowExpressions) && child.maxRows.forall(_ > limit) && limit < conf.topKSortFallbackThreshold => // Sort is needed here because we need global sort. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeOneRowPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeOneRowPlan.scala index 8e066d1cd6340..6f732b2d0f20a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeOneRowPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeOneRowPlan.scala @@ -46,9 +46,9 @@ object OptimizeOneRowPlan extends Rule[LogicalPlan] { val enableForStreaming = conf.getConf(SQLConf.STREAMING_OPTIMIZE_ONE_ROW_PLAN_ENABLED) plan.transformUpWithPruning(_.containsAnyPattern(SORT, AGGREGATE), ruleId) { - case Sort(_, _, child) if child.maxRows.exists(_ <= 1L) && + case Sort(_, _, child, _) if child.maxRows.exists(_ <= 1L) && isChildEligible(child, enableForStreaming) => child - case Sort(_, false, child) if child.maxRowsPerPartition.exists(_ <= 1L) && + case Sort(_, false, child, _) if child.maxRowsPerPartition.exists(_ <= 1L) && isChildEligible(child, enableForStreaming) => child case agg @ Aggregate(_, _, child, _) if agg.groupOnly && child.maxRows.exists(_ <= 1L) && isChildEligible(child, enableForStreaming) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 90d9bd5d5d88e..29216523fefc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -334,7 +334,7 @@ abstract class Optimizer(catalogManager: CatalogManager) return plan } plan match { - case Sort(_, _, child) => child + case Sort(_, _, child, _) => child case Project(fields, child) => Project(fields, removeTopLevelSort(child)) case other => other } @@ -1303,7 +1303,7 @@ object CollapseRepartition extends Rule[LogicalPlan] { // Case 2: When a RepartitionByExpression has a child of global Sort, Repartition or // RepartitionByExpression we can remove the child. case r @ RepartitionByExpression( - _, child @ (Sort(_, true, _) | _: RepartitionOperation), _, _) => + _, child @ (Sort(_, true, _, _) | _: RepartitionOperation), _, _) => r.withNewChildren(child.children) // Case 3: When a RebalancePartitions has a child of local or global Sort, Repartition or // RepartitionByExpression we can remove the child. @@ -1370,11 +1370,11 @@ object CollapseWindow extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning( _.containsPattern(WINDOW), ruleId) { - case w1 @ Window(we1, _, _, w2 @ Window(we2, _, _, grandChild)) + case w1 @ Window(we1, _, _, w2 @ Window(we2, _, _, grandChild, _), _) if windowsCompatible(w1, w2) => w1.copy(windowExpressions = we2 ++ we1, child = grandChild) - case w1 @ Window(we1, _, _, Project(pl, w2 @ Window(we2, _, _, grandChild))) + case w1 @ Window(we1, _, _, Project(pl, w2 @ Window(we2, _, _, grandChild, _)), _) if windowsCompatible(w1, w2) && w1.references.subsetOf(grandChild.outputSet) => Project( pl ++ w1.windowOutputSet, @@ -1403,11 +1403,11 @@ object TransposeWindow extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning( _.containsPattern(WINDOW), ruleId) { - case w1 @ Window(_, _, _, w2 @ Window(_, _, _, grandChild)) + case w1 @ Window(_, _, _, w2 @ Window(_, _, _, grandChild, _), _) if windowsCompatible(w1, w2) => Project(w1.output, w2.copy(child = w1.copy(child = grandChild))) - case w1 @ Window(_, _, _, Project(pl, w2 @ Window(_, _, _, grandChild))) + case w1 @ Window(_, _, _, Project(pl, w2 @ Window(_, _, _, grandChild, _)), _) if windowsCompatible(w1, w2) && w1.references.subsetOf(grandChild.outputSet) => Project( pl ++ w1.windowOutputSet, @@ -1649,14 +1649,14 @@ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { */ object EliminateSorts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transformUpWithPruning(_.containsPattern(SORT)) { - case s @ Sort(orders, _, child) if orders.isEmpty || orders.exists(_.child.foldable) => + case s @ Sort(orders, _, child, _) if orders.isEmpty || orders.exists(_.child.foldable) => val newOrders = orders.filterNot(_.child.foldable) if (newOrders.isEmpty) { child } else { s.copy(order = newOrders) } - case s @ Sort(_, global, child) => s.copy(child = recursiveRemoveSort(child, global)) + case s @ Sort(_, global, child, _) => s.copy(child = recursiveRemoveSort(child, global)) case j @ Join(originLeft, originRight, _, cond, _) if cond.forall(_.deterministic) => j.copy(left = recursiveRemoveSort(originLeft, true), right = recursiveRemoveSort(originRight, true)) @@ -1675,7 +1675,7 @@ object EliminateSorts extends Rule[LogicalPlan] { return plan } plan match { - case Sort(_, global, child) if canRemoveGlobalSort || !global => + case Sort(_, global, child, _) if canRemoveGlobalSort || !global => recursiveRemoveSort(child, canRemoveGlobalSort) case other if canEliminateSort(other) => other.withNewChildren(other.children.map(c => recursiveRemoveSort(c, canRemoveGlobalSort))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantSorts.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantSorts.scala index 204d2a34675bc..3923b9b1b7fae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantSorts.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantSorts.scala @@ -36,14 +36,14 @@ object RemoveRedundantSorts extends Rule[LogicalPlan] { return plan } plan match { - case s @ Sort(orders, false, child) => + case s @ Sort(orders, false, child, _) => if (SortOrder.orderingSatisfies(child.outputOrdering, orders)) { recursiveRemoveSort(child, optimizeGlobalSort = false) } else { s.withNewChildren(Seq(recursiveRemoveSort(child, optimizeGlobalSort = true))) } - case s @ Sort(orders, true, child) => + case s @ Sort(orders, true, child, _) => val newChild = recursiveRemoveSort(child, optimizeGlobalSort = false) if (optimizeGlobalSort) { // For this case, the upper sort is local so the ordering of present sort is unnecessary, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index a666b977030e6..d0a0fc307756c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -364,7 +364,7 @@ object PhysicalWindow { (WindowFunctionType, Seq[NamedExpression], Seq[Expression], Seq[SortOrder], LogicalPlan) def unapply(a: Any): Option[ReturnType] = a match { - case expr @ logical.Window(windowExpressions, partitionSpec, orderSpec, child) => + case expr @ logical.Window(windowExpressions, partitionSpec, orderSpec, child, _) => // The window expression should not be empty here, otherwise it's a bug. if (windowExpressions.isEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index dc286183ac689..0cb04064a6178 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -946,7 +946,8 @@ case class WithWindowDefinition( case class Sort( order: Seq[SortOrder], global: Boolean, - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan, + hint: Option[SortHint] = None) extends UnaryNode { override def output: Seq[Attribute] = child.output override def maxRows: Option[Long] = child.maxRows override def maxRowsPerPartition: Option[Long] = { @@ -1266,7 +1267,8 @@ case class Window( windowExpressions: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan, + hint: Option[WindowHint] = None) extends UnaryNode { override def maxRows: Option[Long] = child.maxRows override def output: Seq[Attribute] = child.output ++ windowExpressions.map(_.toAttribute) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala index 82260755977f0..c8d2be2987457 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/hints.scala @@ -199,6 +199,10 @@ case object NO_BROADCAST_AND_REPLICATION extends JoinStrategyHint { abstract class AggregateHint; +abstract class WindowHint; + +abstract class SortHint; + /** * The callback for implementing customized strategies of handling hint errors. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 96f3ddb72f054..22082aca81a22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -106,28 +106,28 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { private def planTakeOrdered(plan: LogicalPlan): Option[SparkPlan] = plan match { // We should match the combination of limit and offset first, to get the optimal physical // plan, instead of planning limit and offset separately. - case LimitAndOffset(limit, offset, Sort(order, true, child)) + case LimitAndOffset(limit, offset, Sort(order, true, child, _)) if limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( limit, order, child.output, planLater(child), offset)) - case LimitAndOffset(limit, offset, Project(projectList, Sort(order, true, child))) + case LimitAndOffset(limit, offset, Project(projectList, Sort(order, true, child, _))) if limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( limit, order, projectList, planLater(child), offset)) // 'Offset a' then 'Limit b' is the same as 'Limit a + b' then 'Offset a'. - case OffsetAndLimit(offset, limit, Sort(order, true, child)) + case OffsetAndLimit(offset, limit, Sort(order, true, child, _)) if offset + limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( offset + limit, order, child.output, planLater(child), offset)) - case OffsetAndLimit(offset, limit, Project(projectList, Sort(order, true, child))) + case OffsetAndLimit(offset, limit, Project(projectList, Sort(order, true, child, _))) if offset + limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( offset + limit, order, projectList, planLater(child), offset)) - case Limit(IntegerLiteral(limit), Sort(order, true, child)) + case Limit(IntegerLiteral(limit), Sort(order, true, child, _)) if limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( limit, order, child.output, planLater(child))) - case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) + case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child, _))) if limit < conf.topKSortFallbackThreshold => Some(TakeOrderedAndProjectExec( limit, order, projectList, planLater(child))) @@ -978,7 +978,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } - case logical.Sort(sortExprs, global, child) => + case logical.Sort(sortExprs, global, child, _) => execution.SortExec(sortExprs, global, planLater(child)) :: Nil case logical.Project(projectList, child) => execution.ProjectExec(projectList, planLater(child)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala index 2b6fcd9d547f1..23b2647b62a19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2ScanRelationPushDown.scala @@ -420,7 +420,8 @@ object V2ScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper { sHolder.pushedLimit = Some(limit) } (operation, isPushed && !isPartiallyPushed) - case s @ Sort(order, _, operation @ PhysicalOperation(project, Nil, sHolder: ScanBuilderHolder)) + case s @ Sort(order, _, + operation @ PhysicalOperation(project, Nil, sHolder: ScanBuilderHolder), _) if CollapseProject.canCollapseExpressions(order, project, alwaysInline = true) => val aliasMap = getAliasMap(project) val aliasReplacedOrder = order.map(replaceAlias(_, aliasMap)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 0872efd92002c..7daf2c6b1b58b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -114,7 +114,7 @@ trait SQLQueryTestHelper extends Logging { | _: DescribeColumnCommand | _: DescribeRelation | _: DescribeColumn => true - case PhysicalOperation(_, _, Sort(_, true, _)) => true + case PhysicalOperation(_, _, Sort(_, true, _, _)) => true case _ => plan.children.iterator.exists(isSemanticallySorted) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 5ec557462de10..8750c398cc942 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, Max, Partial} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, CompoundBody, ParserInterface} import org.apache.spark.sql.catalyst.plans.{PlanTest, SQLHelper} -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AggregateHint, ColumnStat, Limit, LocalRelation, LogicalPlan, Statistics, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AggregateHint, ColumnStat, Limit, LocalRelation, LogicalPlan, Sort, SortHint, Statistics, UnresolvedHint} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag @@ -559,6 +559,17 @@ class SparkSessionExtensionSuite extends SparkFunSuite with SQLHelper with Adapt compareExpressions(expectedAlias, res.asInstanceOf[Aggregate].aggregateExpressions.head) } } + + test("custom sort hint") { + // The custom hint allows us to replace the sort with its input + withSession(Seq(_.injectHintResolutionRule(CustomerSortHintResolutionRule), + _.injectOptimizerRule(CustomSortRule))) { session => + val res = session.range(10).sort("id") + .hint("INPUT_SORTED") + .queryExecution.optimizedPlan + assert(res.collect {case s: Sort => s}.isEmpty) + } + } } case class MyRule(spark: SparkSession) extends Rule[LogicalPlan] { @@ -1302,3 +1313,27 @@ case class CustomAggregateRule(spark: SparkSession) extends Rule[LogicalPlan] { } } } + +// Example of a Sort hint that tells that the input is already sorted, +// and the rule that removes all Sort nodes based on such hint. +case class CustomSortHint(inputSorted: Boolean) extends SortHint + +// Attaches the CustomSortHint to the sort node. +case class CustomerSortHintResolutionRule(spark: SparkSession) extends Rule[LogicalPlan] { + val MY_HINT_NAME = Set("INPUT_SORTED") + + private def applySortHint(plan: LogicalPlan): LogicalPlan = plan.transformDown { + case s @ Sort(_, _, _, None) => s.copy(hint = Some(CustomSortHint(true))) + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case h: UnresolvedHint if MY_HINT_NAME.contains(h.name.toUpperCase(Locale.ROOT)) => + applySortHint(h.child) + } +} + +case class CustomSortRule(spark: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + case s @ Sort(_, _, _, Some(CustomSortHint(true))) => s.child + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala index a6bf95be837da..8c20a40fede72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDTFSuite.scala @@ -221,7 +221,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { _, false, RepartitionByExpression( _, Project( _, SubqueryAlias( - _, _: LocalRelation)), _, _)) => + _, _: LocalRelation)), _, _), _) => case other => failure(other) } @@ -235,7 +235,7 @@ class PythonUDTFSuite extends QueryTest with SharedSparkSession { case Sort( _, false, Repartition( 1, true, SubqueryAlias( - _, _: LocalRelation))) => + _, _: LocalRelation)), _) => case other => failure(other) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 87e58bb8fa13a..73dda42568a71 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -162,7 +162,7 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case PhysicalOperation(_, _, Sort(_, true, _)) => true + case PhysicalOperation(_, _, Sort(_, true, _, _)) => true case _ => plan.children.iterator.exists(isSorted) } From 8b2d0328c21352e24e4e1c2bbe0a4e97da6a8f2d Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 18 Nov 2024 17:31:33 +0800 Subject: [PATCH 03/58] [SPARK-45265][SQL][BUILD][FOLLOWUP] Add `-Xss64m` for Maven testing of the `sql/hive` module ### What changes were proposed in this pull request? This pr add `-Xss64m` for Maven testing of the `sql/hive` module ### Why are the changes needed? Use test options consistent with sbt and fix the mavren daily test - https://github.com/apache/spark/actions/runs/11879474992/job/33101447051 - https://github.com/apache/spark/actions/runs/11879098676/job/33100585352 - https://github.com/apache/spark/actions/runs/11881833077/job/33106633785 ``` *** RUN ABORTED *** An exception or error caused a run to abort: MetaException(message:Exception rolling back to savepoint rollback_5974738954110) org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:Exception rolling back to savepoint rollback_5974738954110) at org.apache.hadoop.hive.ql.metadata.Hive.addPartitions(Hive.java:3592) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:569) at org.apache.spark.sql.hive.client.Shim_v4_0.createPartitions(HiveShim.scala:1437) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$createPartitions$1(HiveClientImpl.scala:663) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:302) at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:233) ... Cause: org.apache.hadoop.hive.metastore.api.MetaException: Exception rolling back to savepoint rollback_5974738954110 at org.apache.hadoop.hive.metastore.ObjectStore$GetHelper.run(ObjectStore.java:4508) at org.apache.hadoop.hive.metastore.ObjectStore.addPartitionsInternal(ObjectStore.java:2811) at org.apache.hadoop.hive.metastore.ObjectStore.addPartitions(ObjectStore.java:2729) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:569) at org.apache.hadoop.hive.metastore.RawStoreProxy.invoke(RawStoreProxy.java:97) at jdk.proxy25/jdk.proxy25.$Proxy252.addPartitions(Unknown Source) at org.apache.hadoop.hive.metastore.HMSHandler.add_partitions_core(HMSHandler.java:4304) ... ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - https://github.com/LuciferYang/spark/runs/33114760431 ### Was this patch authored or co-authored using generative AI tooling? No Closes #48870 from LuciferYang/SPARK-45265-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Kent Yao --- project/SparkBuild.scala | 2 ++ sql/hive/pom.xml | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f95a95a72b8d5..48b243618eea3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1190,6 +1190,8 @@ object Hive { // smaller size of Xss to mock a FAILED_TO_PARSE_TOO_COMPLEX error, so we need to set for // hive moudle specifically. (Test / javaOptions) := (Test / javaOptions).value.filterNot(_.contains("Xss")), + // SPARK-45265: The value for `-Xss` should be consistent with the configuration value for + // `scalatest-maven-plugin` in `sql/hive/pom.xml` (Test / javaOptions) += "-Xss64m", // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings // only for this subproject. diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5a325f5f56bfc..dc0f88beeab85 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -237,7 +237,11 @@ scalatest-maven-plugin - -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} + + -da -Xmx4g -Xss64m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} From 05750def2da6fe16fc0cf8a3eee79bb2056df979 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 18 Nov 2024 19:18:46 +0800 Subject: [PATCH 04/58] [MINOR][PYTHON][DOCS] Fix the type hint of `histogram_numeric` ### What changes were proposed in this pull request? Fix the type hint of `histogram_numeric` ### Why are the changes needed? argument `nBins` cannot be a column name: ``` In [11]: spark.range(100).withColumn("nb", sf.lit(5)).select(sf.histogram_numeric('id', "nb")).show(truncate=False) ... AnalysisException: [DATATYPE_MISMATCH.NON_FOLDABLE_INPUT] Cannot resolve "histogram_numeric(id, nb)" due to data type mismatch: the input `nb` should be a foldable "INT" expression; however, got "nb". SQLSTATE: 42K09; 'Aggregate [unresolvedalias(histogram_numeric(id#323L, nb#324, 0, 0))] +- Project [id#323L, 5 AS nb#324] +- Range (0, 100, step=1, splits=Some(12)) ``` ### Does this PR introduce _any_ user-facing change? doc-only ### How was this patch tested? updated doctest ### Was this patch authored or co-authored using generative AI tooling? no Closes #48875 from zhengruifeng/fix_histogram_numeric. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../pyspark/sql/connect/functions/builtin.py | 2 +- python/pyspark/sql/functions/builtin.py | 25 ++++++++----------- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index fa1a81ab04eba..f52cdffb84b7c 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -1552,7 +1552,7 @@ def count_if(col: "ColumnOrName") -> Column: count_if.__doc__ = pysparkfuncs.count_if.__doc__ -def histogram_numeric(col: "ColumnOrName", nBins: "ColumnOrName") -> Column: +def histogram_numeric(col: "ColumnOrName", nBins: Column) -> Column: return _invoke_function_over_columns("histogram_numeric", col, nBins) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 842fdf7ae4234..1beaa4807ef33 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -8557,7 +8557,7 @@ def count_if(col: "ColumnOrName") -> Column: @_try_remote_functions -def histogram_numeric(col: "ColumnOrName", nBins: "ColumnOrName") -> Column: +def histogram_numeric(col: "ColumnOrName", nBins: Column) -> Column: """Computes a histogram on numeric 'col' using nb bins. The return value is an array of (x,y) pairs representing the centers of the histogram's bins. As the value of 'nb' is increased, the histogram approximation @@ -8573,9 +8573,9 @@ def histogram_numeric(col: "ColumnOrName", nBins: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. - nBins : :class:`~pyspark.sql.Column` or str + nBins : :class:`~pyspark.sql.Column` number of Histogram columns. Returns @@ -8585,17 +8585,14 @@ def histogram_numeric(col: "ColumnOrName", nBins: "ColumnOrName") -> Column: Examples -------- - >>> df = spark.createDataFrame([("a", 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) - >>> df.select(histogram_numeric('c2', lit(5))).show() - +------------------------+ - |histogram_numeric(c2, 5)| - +------------------------+ - | [{1, 1.0}, {2, 1....| - +------------------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.range(100, numPartitions=1) + >>> df.select(sf.histogram_numeric('id', sf.lit(5))).show(truncate=False) + +-----------------------------------------------------------+ + |histogram_numeric(id, 5) | + +-----------------------------------------------------------+ + |[{11, 25.0}, {36, 24.0}, {59, 23.0}, {84, 25.0}, {98, 3.0}]| + +-----------------------------------------------------------+ """ return _invoke_function_over_columns("histogram_numeric", col, nBins) From 400a8d3797bdcc9183576e66e84163e4dc00a662 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 18 Nov 2024 22:49:32 +0800 Subject: [PATCH 05/58] Revert "[SPARK-49787][SQL] Cast between UDT and other types" This reverts commit b6681fbf32fa3596d7649d413f20cc5c6da64991. --- python/pyspark/sql/tests/test_types.py | 16 +- .../apache/spark/sql/types/UpCastRule.scala | 4 - .../spark/sql/catalyst/expressions/Cast.scala | 175 ++++++++---------- .../sql/catalyst/expressions/literals.scala | 84 ++++----- .../catalyst/expressions/CastSuiteBase.scala | 42 +---- 5 files changed, 119 insertions(+), 202 deletions(-) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 9688ed4923737..8a30608cd4087 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -28,6 +28,7 @@ from pyspark.sql import Row from pyspark.sql import functions as F from pyspark.errors import ( + AnalysisException, ParseException, PySparkTypeError, PySparkValueError, @@ -1129,17 +1130,10 @@ def test_cast_to_string_with_udt(self): def test_cast_to_udt_with_udt(self): row = Row(point=ExamplePoint(1.0, 2.0), python_only_point=PythonOnlyPoint(1.0, 2.0)) df = self.spark.createDataFrame([row]) - result = df.select(F.col("point").cast(PythonOnlyUDT())).collect() - self.assertEqual( - result, - [Row(point=PythonOnlyPoint(1.0, 2.0))], - ) - - result = df.select(F.col("python_only_point").cast(ExamplePointUDT())).collect() - self.assertEqual( - result, - [Row(python_only_point=ExamplePoint(1.0, 2.0))], - ) + with self.assertRaises(AnalysisException): + df.select(F.col("point").cast(PythonOnlyUDT())).collect() + with self.assertRaises(AnalysisException): + df.select(F.col("python_only_point").cast(ExamplePointUDT())).collect() def test_struct_type(self): struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala index 6f2fd41f1f799..4993e249b3059 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/UpCastRule.scala @@ -66,10 +66,6 @@ private[sql] object UpCastRule { case (from: UserDefinedType[_], to: UserDefinedType[_]) if to.acceptsType(from) => true - case (udt: UserDefinedType[_], toType) => canUpCast(udt.sqlType, toType) - - case (fromType, udt: UserDefinedType[_]) => canUpCast(fromType, udt.sqlType) - case _ => false } 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 b1e3a4ad21e41..154199d37c46d 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 @@ -150,10 +150,6 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true - case (udt: UserDefinedType[_], toType) => canAnsiCast(udt.sqlType, toType) - - case (fromType, udt: UserDefinedType[_]) => canAnsiCast(fromType, udt.sqlType) - case _ => false } @@ -271,10 +267,6 @@ object Cast extends QueryErrorsBase { case (udt1: UserDefinedType[_], udt2: UserDefinedType[_]) if udt2.acceptsType(udt1) => true - case (udt: UserDefinedType[_], toType) => canCast(udt.sqlType, toType) - - case (fromType, udt: UserDefinedType[_]) => canCast(fromType, udt.sqlType) - case _ => false } @@ -1131,42 +1123,33 @@ case class Cast( variant.VariantGet.cast(v, to, evalMode != EvalMode.TRY, timeZoneId, zoneId) }) } else { - from match { - // `castToString` has special handling for `UserDefinedType` - case udt: UserDefinedType[_] if !to.isInstanceOf[StringType] => - castInternal(udt.sqlType, to) - case _ => - to match { - case dt if dt == from => identity[Any] - case VariantType => input => - variant.VariantExpressionEvalUtils.castToVariant(input, from) - case _: StringType => castToString(from) - case BinaryType => castToBinary(from) - case DateType => castToDate(from) - case decimal: DecimalType => castToDecimal(from, decimal) - case TimestampType => castToTimestamp(from) - case TimestampNTZType => castToTimestampNTZ(from) - case CalendarIntervalType => castToInterval(from) - case it: DayTimeIntervalType => castToDayTimeInterval(from, it) - case it: YearMonthIntervalType => castToYearMonthInterval(from, it) - case BooleanType => castToBoolean(from) - case ByteType => castToByte(from) - case ShortType => castToShort(from) - case IntegerType => castToInt(from) - case FloatType => castToFloat(from) - case LongType => castToLong(from) - case DoubleType => castToDouble(from) - case array: ArrayType => - castArray(from.asInstanceOf[ArrayType].elementType, array.elementType) - case map: MapType => castMap(from.asInstanceOf[MapType], map) - case struct: StructType => castStruct(from.asInstanceOf[StructType], struct) - case udt: UserDefinedType[_] if udt.acceptsType(from) => - identity[Any] - case udt: UserDefinedType[_] => - castInternal(from, udt.sqlType) - case _ => - throw QueryExecutionErrors.cannotCastError(from, to) - } + to match { + case dt if dt == from => identity[Any] + case VariantType => input => variant.VariantExpressionEvalUtils.castToVariant(input, from) + case _: StringType => castToString(from) + case BinaryType => castToBinary(from) + case DateType => castToDate(from) + case decimal: DecimalType => castToDecimal(from, decimal) + case TimestampType => castToTimestamp(from) + case TimestampNTZType => castToTimestampNTZ(from) + case CalendarIntervalType => castToInterval(from) + case it: DayTimeIntervalType => castToDayTimeInterval(from, it) + case it: YearMonthIntervalType => castToYearMonthInterval(from, it) + case BooleanType => castToBoolean(from) + case ByteType => castToByte(from) + case ShortType => castToShort(from) + case IntegerType => castToInt(from) + case FloatType => castToFloat(from) + case LongType => castToLong(from) + case DoubleType => castToDouble(from) + case array: ArrayType => + castArray(from.asInstanceOf[ArrayType].elementType, array.elementType) + case map: MapType => castMap(from.asInstanceOf[MapType], map) + case struct: StructType => castStruct(from.asInstanceOf[StructType], struct) + case udt: UserDefinedType[_] if udt.acceptsType(from) => + identity[Any] + case _: UserDefinedType[_] => + throw QueryExecutionErrors.cannotCastError(from, to) } } } @@ -1228,64 +1211,54 @@ case class Cast( private[this] def nullSafeCastFunction( from: DataType, to: DataType, - ctx: CodegenContext): CastFunction = { - from match { - // `castToStringCode` has special handling for `UserDefinedType` - case udt: UserDefinedType[_] if !to.isInstanceOf[StringType] => - nullSafeCastFunction(udt.sqlType, to, ctx) - case _ => - to match { - - case _ if from == NullType => (c, evPrim, evNull) => code"$evNull = true;" - case _ if to == from => (c, evPrim, evNull) => code"$evPrim = $c;" - case _ if from.isInstanceOf[VariantType] => (c, evPrim, evNull) => - val tmp = ctx.freshVariable("tmp", classOf[Object]) - val dataTypeArg = ctx.addReferenceObj("dataType", to) - val zoneStrArg = ctx.addReferenceObj("zoneStr", timeZoneId) - val zoneIdArg = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) - val failOnError = evalMode != EvalMode.TRY - val cls = classOf[variant.VariantGet].getName - code""" - Object $tmp = $cls.cast($c, $dataTypeArg, $failOnError, $zoneStrArg, $zoneIdArg); - if ($tmp == null) { - $evNull = true; - } else { - $evPrim = (${CodeGenerator.boxedType(to)})$tmp; - } - """ - case VariantType => - val cls = variant.VariantExpressionEvalUtils.getClass.getName.stripSuffix("$") - val fromArg = ctx.addReferenceObj("from", from) - (c, evPrim, evNull) => code"$evPrim = $cls.castToVariant($c, $fromArg);" - case _: StringType => (c, evPrim, _) => castToStringCode(from, ctx).apply(c, evPrim) - case BinaryType => castToBinaryCode(from) - case DateType => castToDateCode(from, ctx) - case decimal: DecimalType => castToDecimalCode(from, decimal, ctx) - case TimestampType => castToTimestampCode(from, ctx) - case TimestampNTZType => castToTimestampNTZCode(from, ctx) - case CalendarIntervalType => castToIntervalCode(from) - case it: DayTimeIntervalType => castToDayTimeIntervalCode(from, it) - case it: YearMonthIntervalType => castToYearMonthIntervalCode(from, it) - case BooleanType => castToBooleanCode(from, ctx) - case ByteType => castToByteCode(from, ctx) - case ShortType => castToShortCode(from, ctx) - case IntegerType => castToIntCode(from, ctx) - case FloatType => castToFloatCode(from, ctx) - case LongType => castToLongCode(from, ctx) - case DoubleType => castToDoubleCode(from, ctx) - - case array: ArrayType => - castArrayCode(from.asInstanceOf[ArrayType].elementType, array.elementType, ctx) - case map: MapType => castMapCode(from.asInstanceOf[MapType], map, ctx) - case struct: StructType => castStructCode(from.asInstanceOf[StructType], struct, ctx) - case udt: UserDefinedType[_] if udt.acceptsType(from) => - (c, evPrim, evNull) => code"$evPrim = $c;" - case udt: UserDefinedType[_] => - nullSafeCastFunction(from, udt.sqlType, ctx) - case _ => - throw QueryExecutionErrors.cannotCastError(from, to) + ctx: CodegenContext): CastFunction = to match { + + case _ if from == NullType => (c, evPrim, evNull) => code"$evNull = true;" + case _ if to == from => (c, evPrim, evNull) => code"$evPrim = $c;" + case _ if from.isInstanceOf[VariantType] => (c, evPrim, evNull) => + val tmp = ctx.freshVariable("tmp", classOf[Object]) + val dataTypeArg = ctx.addReferenceObj("dataType", to) + val zoneStrArg = ctx.addReferenceObj("zoneStr", timeZoneId) + val zoneIdArg = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val failOnError = evalMode != EvalMode.TRY + val cls = classOf[variant.VariantGet].getName + code""" + Object $tmp = $cls.cast($c, $dataTypeArg, $failOnError, $zoneStrArg, $zoneIdArg); + if ($tmp == null) { + $evNull = true; + } else { + $evPrim = (${CodeGenerator.boxedType(to)})$tmp; } - } + """ + case VariantType => + val cls = variant.VariantExpressionEvalUtils.getClass.getName.stripSuffix("$") + val fromArg = ctx.addReferenceObj("from", from) + (c, evPrim, evNull) => code"$evPrim = $cls.castToVariant($c, $fromArg);" + case _: StringType => (c, evPrim, _) => castToStringCode(from, ctx).apply(c, evPrim) + case BinaryType => castToBinaryCode(from) + case DateType => castToDateCode(from, ctx) + case decimal: DecimalType => castToDecimalCode(from, decimal, ctx) + case TimestampType => castToTimestampCode(from, ctx) + case TimestampNTZType => castToTimestampNTZCode(from, ctx) + case CalendarIntervalType => castToIntervalCode(from) + case it: DayTimeIntervalType => castToDayTimeIntervalCode(from, it) + case it: YearMonthIntervalType => castToYearMonthIntervalCode(from, it) + case BooleanType => castToBooleanCode(from, ctx) + case ByteType => castToByteCode(from, ctx) + case ShortType => castToShortCode(from, ctx) + case IntegerType => castToIntCode(from, ctx) + case FloatType => castToFloatCode(from, ctx) + case LongType => castToLongCode(from, ctx) + case DoubleType => castToDoubleCode(from, ctx) + + case array: ArrayType => + castArrayCode(from.asInstanceOf[ArrayType].elementType, array.elementType, ctx) + case map: MapType => castMapCode(from.asInstanceOf[MapType], map, ctx) + case struct: StructType => castStructCode(from.asInstanceOf[StructType], struct, ctx) + case udt: UserDefinedType[_] if udt.acceptsType(from) => + (c, evPrim, evNull) => code"$evPrim = $c;" + case _: UserDefinedType[_] => + throw QueryExecutionErrors.cannotCastError(from, to) } // Since we need to cast input expressions recursively inside ComplexTypes, such as Map's diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 362bb9af1661e..4cffc7f0b53a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -441,53 +441,47 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { override def eval(input: InternalRow): Any = value override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - def gen(ctx: CodegenContext, ev: ExprCode, dataType: DataType): ExprCode = { - val javaType = CodeGenerator.javaType(dataType) - if (value == null) { - ExprCode.forNullValue(dataType) - } else { - def toExprCode(code: String): ExprCode = { - ExprCode.forNonNullValue(JavaCode.literal(code, dataType)) - } - - dataType match { - case BooleanType | IntegerType | DateType | _: YearMonthIntervalType => - toExprCode(value.toString) - case FloatType => - value.asInstanceOf[Float] match { - case v if v.isNaN => - toExprCode("Float.NaN") - case Float.PositiveInfinity => - toExprCode("Float.POSITIVE_INFINITY") - case Float.NegativeInfinity => - toExprCode("Float.NEGATIVE_INFINITY") - case _ => - toExprCode(s"${value}F") - } - case DoubleType => - value.asInstanceOf[Double] match { - case v if v.isNaN => - toExprCode("Double.NaN") - case Double.PositiveInfinity => - toExprCode("Double.POSITIVE_INFINITY") - case Double.NegativeInfinity => - toExprCode("Double.NEGATIVE_INFINITY") - case _ => - toExprCode(s"${value}D") - } - case ByteType | ShortType => - ExprCode.forNonNullValue(JavaCode.expression(s"($javaType)$value", dataType)) - case TimestampType | TimestampNTZType | LongType | _: DayTimeIntervalType => - toExprCode(s"${value}L") - case udt: UserDefinedType[_] => - gen(ctx, ev, udt.sqlType) - case _ => - val constRef = ctx.addReferenceObj("literal", value, javaType) - ExprCode.forNonNullValue(JavaCode.global(constRef, dataType)) - } + val javaType = CodeGenerator.javaType(dataType) + if (value == null) { + ExprCode.forNullValue(dataType) + } else { + def toExprCode(code: String): ExprCode = { + ExprCode.forNonNullValue(JavaCode.literal(code, dataType)) + } + dataType match { + case BooleanType | IntegerType | DateType | _: YearMonthIntervalType => + toExprCode(value.toString) + case FloatType => + value.asInstanceOf[Float] match { + case v if v.isNaN => + toExprCode("Float.NaN") + case Float.PositiveInfinity => + toExprCode("Float.POSITIVE_INFINITY") + case Float.NegativeInfinity => + toExprCode("Float.NEGATIVE_INFINITY") + case _ => + toExprCode(s"${value}F") + } + case DoubleType => + value.asInstanceOf[Double] match { + case v if v.isNaN => + toExprCode("Double.NaN") + case Double.PositiveInfinity => + toExprCode("Double.POSITIVE_INFINITY") + case Double.NegativeInfinity => + toExprCode("Double.NEGATIVE_INFINITY") + case _ => + toExprCode(s"${value}D") + } + case ByteType | ShortType => + ExprCode.forNonNullValue(JavaCode.expression(s"($javaType)$value", dataType)) + case TimestampType | TimestampNTZType | LongType | _: DayTimeIntervalType => + toExprCode(s"${value}L") + case _ => + val constRef = ctx.addReferenceObj("literal", value, javaType) + ExprCode.forNonNullValue(JavaCode.global(constRef, dataType)) } } - gen(ctx, ev, dataType) } override def sql: String = (value, dataType) match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala index f915d6efeb827..e87b54339821f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuiteBase.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} -import java.time.{Duration, LocalDate, LocalDateTime, Period, Year => JYear} +import java.time.{Duration, LocalDate, LocalDateTime, Period} import java.time.temporal.ChronoUnit import java.util.{Calendar, Locale, TimeZone} @@ -37,7 +37,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DataTypeTestUtils.{dayTimeIntervalTypes, yearMonthIntervalTypes} import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, MINUTE, SECOND} -import org.apache.spark.sql.types.TestUDT._ import org.apache.spark.sql.types.UpCastRule.numericPrecedence import org.apache.spark.sql.types.YearMonthIntervalType.{MONTH, YEAR} import org.apache.spark.unsafe.types.UTF8String @@ -1410,43 +1409,4 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { assert(!Cast(timestampLiteral, TimestampNTZType).resolved) assert(!Cast(timestampNTZLiteral, TimestampType).resolved) } - - test("SPARK-49787: Cast between UDT and other types") { - val value = new MyDenseVector(Array(1.0, 2.0, -1.0)) - val udtType = new MyDenseVectorUDT() - val targetType = ArrayType(DoubleType, containsNull = false) - - val serialized = udtType.serialize(value) - - checkEvaluation(Cast(new Literal(serialized, udtType), targetType), serialized) - checkEvaluation(Cast(new Literal(serialized, targetType), udtType), serialized) - - val year = JYear.parse("2024") - val yearUDTType = new YearUDT() - - val yearSerialized = yearUDTType.serialize(year) - - checkEvaluation(Cast(new Literal(yearSerialized, yearUDTType), IntegerType), 2024) - checkEvaluation(Cast(new Literal(2024, IntegerType), yearUDTType), yearSerialized) - - val yearString = UTF8String.fromString("2024") - checkEvaluation(Cast(new Literal(yearSerialized, yearUDTType), StringType), yearString) - checkEvaluation(Cast(new Literal(yearString, StringType), yearUDTType), yearSerialized) - } -} - -private[sql] class YearUDT extends UserDefinedType[JYear] { - override def sqlType: DataType = IntegerType - - override def serialize(obj: JYear): Int = { - obj.getValue - } - - def deserialize(datum: Any): JYear = datum match { - case value: Int => JYear.of(value) - } - - override def userClass: Class[JYear] = classOf[JYear] - - private[spark] override def asNullable: YearUDT = this } From fa36e8b6080aa202f5f76dffc025e64378da9716 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 19 Nov 2024 09:13:38 +0900 Subject: [PATCH 06/58] [SPARK-50335][PYTHON][DOCS] Refine docstrings for window/aggregation functions - part 2 ### What changes were proposed in this pull request? Refine docstrings for window/aggregation functions: 1, make examples copy-pasteable; 2, specify the meaning of `str` input: either column name or literal string; 3, show both input and output columns; 4, add `see also` section for related functions; ### Why are the changes needed? 1, improve the documentation; 2, improve the doctest coverage; ### Does this PR introduce _any_ user-facing change? doc changes ### How was this patch tested? updated doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48877 from zhengruifeng/py_doc_4. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 503 ++++++++++++++---------- 1 file changed, 292 insertions(+), 211 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1beaa4807ef33..5fce4a67f5662 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6078,10 +6078,11 @@ def dense_rank() -> Column: Examples -------- - >>> from pyspark.sql import Window, types - >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], types.IntegerType()) + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql import Window + >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], "int") >>> w = Window.orderBy("value") - >>> df.withColumn("drank", dense_rank().over(w)).show() + >>> df.withColumn("drank", sf.dense_rank().over(w)).show() +-----+-----+ |value|drank| +-----+-----+ @@ -6121,10 +6122,11 @@ def rank() -> Column: Examples -------- - >>> from pyspark.sql import Window, types - >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], types.IntegerType()) + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql import Window + >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], "int") >>> w = Window.orderBy("value") - >>> df.withColumn("drank", rank().over(w)).show() + >>> df.withColumn("drank", sf.rank().over(w)).show() +-----+-----+ |value|drank| +-----+-----+ @@ -6157,10 +6159,11 @@ def cume_dist() -> Column: Examples -------- - >>> from pyspark.sql import Window, types - >>> df = spark.createDataFrame([1, 2, 3, 3, 4], types.IntegerType()) + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql import Window + >>> df = spark.createDataFrame([1, 2, 3, 3, 4], "int") >>> w = Window.orderBy("value") - >>> df.withColumn("cd", cume_dist().over(w)).show() + >>> df.withColumn("cd", sf.cume_dist().over(w)).show() +-----+---+ |value| cd| +-----+---+ @@ -6191,10 +6194,11 @@ def percent_rank() -> Column: Examples -------- - >>> from pyspark.sql import Window, types - >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], types.IntegerType()) + >>> from pyspark.sql import functions as sf + >>> from pyspark.sql import Window + >>> df = spark.createDataFrame([1, 1, 2, 3, 3, 4], "int") >>> w = Window.orderBy("value") - >>> df.withColumn("pr", percent_rank().over(w)).show() + >>> df.withColumn("pr", sf.percent_rank().over(w)).show() +-----+---+ |value| pr| +-----+---+ @@ -6240,7 +6244,7 @@ def approx_count_distinct(col: "ColumnOrName", rsd: Optional[float] = None) -> C Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The label of the column to count distinct values in. rsd : float, optional The maximum allowed relative standard deviation (default = 0.05). @@ -6259,47 +6263,46 @@ def approx_count_distinct(col: "ColumnOrName", rsd: Optional[float] = None) -> C -------- Example 1: Counting distinct values in a single column DataFrame representing integers - >>> from pyspark.sql.functions import approx_count_distinct + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([1,2,2,3], "int") - >>> df.agg(approx_count_distinct("value").alias('distinct_values')).show() - +---------------+ - |distinct_values| - +---------------+ - | 3| - +---------------+ + >>> df.agg(sf.approx_count_distinct("value")).show() + +----------------------------+ + |approx_count_distinct(value)| + +----------------------------+ + | 3| + +----------------------------+ Example 2: Counting distinct values in a single column DataFrame representing strings - >>> from pyspark.sql.functions import approx_count_distinct + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("apple",), ("orange",), ("apple",), ("banana",)], ['fruit']) - >>> df.agg(approx_count_distinct("fruit").alias('distinct_fruits')).show() - +---------------+ - |distinct_fruits| - +---------------+ - | 3| - +---------------+ + >>> df.agg(sf.approx_count_distinct("fruit")).show() + +----------------------------+ + |approx_count_distinct(fruit)| + +----------------------------+ + | 3| + +----------------------------+ Example 3: Counting distinct values in a DataFrame with multiple columns - >>> from pyspark.sql.functions import approx_count_distinct, struct - >>> df = spark.createDataFrame([("Alice", 1), - ... ("Alice", 2), - ... ("Bob", 3), - ... ("Bob", 3)], ["name", "value"]) - >>> df = df.withColumn("combined", struct("name", "value")) - >>> df.agg(approx_count_distinct("combined").alias('distinct_pairs')).show() - +--------------+ - |distinct_pairs| - +--------------+ - | 3| - +--------------+ + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame( + ... [("Alice", 1), ("Alice", 2), ("Bob", 3), ("Bob", 3)], ["name", "value"]) + >>> df = df.withColumn("combined", sf.struct("name", "value")) + >>> df.agg(sf.approx_count_distinct(df.combined)).show() + +-------------------------------+ + |approx_count_distinct(combined)| + +-------------------------------+ + | 3| + +-------------------------------+ Example 4: Counting distinct values with a specified relative standard deviation - >>> from pyspark.sql.functions import approx_count_distinct - >>> df = spark.range(100000) - >>> df.agg(approx_count_distinct("id").alias('with_default_rsd'), - ... approx_count_distinct("id", 0.1).alias('with_rsd_0.1')).show() + >>> from pyspark.sql import functions as sf + >>> spark.range(100000).agg( + ... sf.approx_count_distinct("id").alias('with_default_rsd'), + ... sf.approx_count_distinct("id", 0.1).alias('with_rsd_0.1') + ... ).show() +----------------+------------+ |with_default_rsd|with_rsd_0.1| +----------------+------------+ @@ -6331,10 +6334,10 @@ def broadcast(df: "ParentDataFrame") -> "ParentDataFrame": Examples -------- - >>> from pyspark.sql import types - >>> df = spark.createDataFrame([1, 2, 3, 3, 4], types.IntegerType()) + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([1, 2, 3, 3, 4], "int") >>> df_small = spark.range(3) - >>> df_b = broadcast(df_small) + >>> df_b = sf.broadcast(df_small) >>> df.join(df_b, df.value == df_small.id).show() +-----+---+ |value| id| @@ -6360,7 +6363,7 @@ def coalesce(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name list of columns to work on. Returns @@ -6370,8 +6373,9 @@ def coalesce(*cols: "ColumnOrName") -> Column: Examples -------- - >>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) - >>> cDf.show() + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b")) + >>> df.show() +----+----+ | a| b| +----+----+ @@ -6380,16 +6384,16 @@ def coalesce(*cols: "ColumnOrName") -> Column: |NULL| 2| +----+----+ - >>> cDf.select(coalesce(cDf["a"], cDf["b"])).show() - +--------------+ - |coalesce(a, b)| - +--------------+ - | NULL| - | 1| - | 2| - +--------------+ + >>> df.select('*', sf.coalesce("a", df["b"])).show() + +----+----+--------------+ + | a| b|coalesce(a, b)| + +----+----+--------------+ + |NULL|NULL| NULL| + | 1|NULL| 1| + |NULL| 2| 2| + +----+----+--------------+ - >>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show() + >>> df.select('*', sf.coalesce(df["a"], lit(0.0))).show() +----+----+----------------+ | a| b|coalesce(a, 0.0)| +----+----+----------------+ @@ -6413,9 +6417,9 @@ def corr(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Parameters ---------- - col1 : :class:`~pyspark.sql.Column` or str + col1 : :class:`~pyspark.sql.Column` or column name first column to calculate correlation. - col2 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or column name second column to calculate correlation. Returns @@ -6425,11 +6429,16 @@ def corr(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> a = range(20) >>> b = [2 * x for x in range(20)] >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) - >>> df.agg(corr("a", "b").alias('c')).collect() - [Row(c=1.0)] + >>> df.agg(sf.corr("a", df.b)).show() + +----------+ + |corr(a, b)| + +----------+ + | 1.0| + +----------+ """ return _invoke_function_over_columns("corr", col1, col2) @@ -6446,9 +6455,9 @@ def covar_pop(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Parameters ---------- - col1 : :class:`~pyspark.sql.Column` or str + col1 : :class:`~pyspark.sql.Column` or column name first column to calculate covariance. - col2 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or column name second column to calculate covariance. Returns @@ -6456,13 +6465,22 @@ def covar_pop(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` covariance of these two column values. + See Also + -------- + :meth:`pyspark.sql.functions.covar_samp` + Examples -------- + >>> from pyspark.sql import functions as sf >>> a = [1] * 10 >>> b = [1] * 10 >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) - >>> df.agg(covar_pop("a", "b").alias('c')).collect() - [Row(c=0.0)] + >>> df.agg(sf.covar_pop("a", df.b)).show() + +---------------+ + |covar_pop(a, b)| + +---------------+ + | 0.0| + +---------------+ """ return _invoke_function_over_columns("covar_pop", col1, col2) @@ -6479,9 +6497,9 @@ def covar_samp(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Parameters ---------- - col1 : :class:`~pyspark.sql.Column` or str + col1 : :class:`~pyspark.sql.Column` or column name first column to calculate covariance. - col2 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or column name second column to calculate covariance. Returns @@ -6489,13 +6507,22 @@ def covar_samp(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` sample covariance of these two column values. + See Also + -------- + :meth:`pyspark.sql.functions.covar_pop` + Examples -------- + >>> from pyspark.sql import functions as sf >>> a = [1] * 10 >>> b = [1] * 10 >>> df = spark.createDataFrame(zip(a, b), ["a", "b"]) - >>> df.agg(covar_samp("a", "b").alias('c')).collect() - [Row(c=0.0)] + >>> df.agg(sf.covar_samp("a", df.b)).show() + +----------------+ + |covar_samp(a, b)| + +----------------+ + | 0.0| + +----------------+ """ return _invoke_function_over_columns("covar_samp", col1, col2) @@ -6544,9 +6571,9 @@ def count_distinct(col: "ColumnOrName", *cols: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name first column to compute on. - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name other columns to compute on. Returns @@ -6616,7 +6643,7 @@ def first(col: "ColumnOrName", ignorenulls: bool = False) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column to fetch first value for. ignorenulls : bool if first value is null then look for first non-null value. ``False``` by default. @@ -6628,9 +6655,10 @@ def first(col: "ColumnOrName", ignorenulls: bool = False) -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", 2), ("Bob", 5), ("Alice", None)], ("name", "age")) >>> df = df.orderBy(df.age) - >>> df.groupby("name").agg(first("age")).orderBy("name").show() + >>> df.groupby("name").agg(sf.first("age")).orderBy("name").show() +-----+----------+ | name|first(age)| +-----+----------+ @@ -6640,7 +6668,7 @@ def first(col: "ColumnOrName", ignorenulls: bool = False) -> Column: To ignore any null values, set ``ignorenulls`` to `True` - >>> df.groupby("name").agg(first("age", ignorenulls=True)).orderBy("name").show() + >>> df.groupby("name").agg(sf.first("age", ignorenulls=True)).orderBy("name").show() +-----+----------+ | name|first(age)| +-----+----------+ @@ -6666,7 +6694,7 @@ def grouping(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column to check if it's aggregated. Returns @@ -6676,8 +6704,9 @@ def grouping(col: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", 2), ("Bob", 5)], ("name", "age")) - >>> df.cube("name").agg(grouping("name"), sum("age")).orderBy("name").show() + >>> df.cube("name").agg(sf.grouping("name"), sf.sum("age")).orderBy("name").show() +-----+--------------+--------+ | name|grouping(name)|sum(age)| +-----+--------------+--------+ @@ -6708,7 +6737,7 @@ def grouping_id(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name columns to check for. Returns @@ -6718,10 +6747,10 @@ def grouping_id(*cols: "ColumnOrName") -> Column: Examples -------- - >>> df = spark.createDataFrame([(1, "a", "a"), - ... (3, "a", "a"), - ... (4, "b", "c")], ["c1", "c2", "c3"]) - >>> df.cube("c2", "c3").agg(grouping_id(), sum("c1")).orderBy("c2", "c3").show() + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame( + ... [(1, "a", "a"), (3, "a", "a"), (4, "b", "c")], ["c1", "c2", "c3"]) + >>> df.cube("c2", "c3").agg(sf.grouping_id(), sf.sum("c1")).orderBy("c2", "c3").show() +----+----+-------------+-------+ | c2| c3|grouping_id()|sum(c1)| +----+----+-------------+-------+ @@ -6754,7 +6783,7 @@ def count_min_sketch( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. eps : :class:`~pyspark.sql.Column` or float relative error, must be positive @@ -6855,9 +6884,10 @@ def input_file_name() -> Column: Examples -------- >>> import os + >>> from pyspark.sql import functions as sf >>> path = os.path.abspath(__file__) >>> df = spark.read.text(path) - >>> df.select(input_file_name()).first() + >>> df.select(sf.input_file_name()).first() Row(input_file_name()='file:///...') """ return _invoke_function("input_file_name") @@ -6874,7 +6904,7 @@ def isnan(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -6884,14 +6914,15 @@ def isnan(col: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) - >>> df.select("a", "b", isnan("a").alias("r1"), isnan(df.b).alias("r2")).show() - +---+---+-----+-----+ - | a| b| r1| r2| - +---+---+-----+-----+ - |1.0|NaN|false| true| - |NaN|2.0| true|false| - +---+---+-----+-----+ + >>> df.select("*", sf.isnan("a"), sf.isnan(df.b)).show() + +---+---+--------+--------+ + | a| b|isnan(a)|isnan(b)| + +---+---+--------+--------+ + |1.0|NaN| false| true| + |NaN|2.0| true| false| + +---+---+--------+--------+ """ return _invoke_function_over_columns("isnan", col) @@ -6907,7 +6938,7 @@ def isnull(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -6917,14 +6948,15 @@ def isnull(col: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b")) - >>> df.select("a", "b", isnull("a").alias("r1"), isnull(df.b).alias("r2")).show() - +----+----+-----+-----+ - | a| b| r1| r2| - +----+----+-----+-----+ - | 1|NULL|false| true| - |NULL| 2| true|false| - +----+----+-----+-----+ + >>> df.select("*", sf.isnull("a"), isnull(df.b)).show() + +----+----+-----------+-----------+ + | a| b|(a IS NULL)|(b IS NULL)| + +----+----+-----------+-----------+ + | 1|NULL| false| true| + |NULL| 2| true| false| + +----+----+-----------+-----------+ """ return _invoke_function_over_columns("isnull", col) @@ -6948,7 +6980,7 @@ def last(col: "ColumnOrName", ignorenulls: bool = False) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column to fetch last value for. ignorenulls : bool if last value is null then look for non-null value. ``False``` by default. @@ -6960,9 +6992,10 @@ def last(col: "ColumnOrName", ignorenulls: bool = False) -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", 2), ("Bob", 5), ("Alice", None)], ("name", "age")) >>> df = df.orderBy(df.age.desc()) - >>> df.groupby("name").agg(last("age")).orderBy("name").show() + >>> df.groupby("name").agg(sf.last("age")).orderBy("name").show() +-----+---------+ | name|last(age)| +-----+---------+ @@ -6972,7 +7005,7 @@ def last(col: "ColumnOrName", ignorenulls: bool = False) -> Column: To ignore any null values, set ``ignorenulls`` to `True` - >>> df.groupby("name").agg(last("age", ignorenulls=True)).orderBy("name").show() + >>> df.groupby("name").agg(sf.last("age", ignorenulls=True)).orderBy("name").show() +-----+---------+ | name|last(age)| +-----+---------+ @@ -7015,21 +7048,24 @@ def monotonically_increasing_id() -> Column: Examples -------- >>> from pyspark.sql import functions as sf - >>> spark.range(0, 10, 1, 2).select(sf.monotonically_increasing_id()).show() - +-----------------------------+ - |monotonically_increasing_id()| - +-----------------------------+ - | 0| - | 1| - | 2| - | 3| - | 4| - | 8589934592| - | 8589934593| - | 8589934594| - | 8589934595| - | 8589934596| - +-----------------------------+ + >>> spark.range(0, 10, 1, 2).select( + ... "*", + ... sf.spark_partition_id(), + ... sf.monotonically_increasing_id()).show() + +---+--------------------+-----------------------------+ + | id|SPARK_PARTITION_ID()|monotonically_increasing_id()| + +---+--------------------+-----------------------------+ + | 0| 0| 0| + | 1| 0| 1| + | 2| 0| 2| + | 3| 0| 3| + | 4| 0| 4| + | 5| 1| 8589934592| + | 6| 1| 8589934593| + | 7| 1| 8589934594| + | 8| 1| 8589934595| + | 9| 1| 8589934596| + +---+--------------------+-----------------------------+ """ return _invoke_function("monotonically_increasing_id") @@ -7047,9 +7083,9 @@ def nanvl(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Parameters ---------- - col1 : :class:`~pyspark.sql.Column` or str + col1 : :class:`~pyspark.sql.Column` or column name first column to check. - col2 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or column name second column to return if first is NaN. Returns @@ -7059,9 +7095,15 @@ def nanvl(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b")) - >>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect() - [Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)] + >>> df.select("*", sf.nanvl("a", "b"), sf.nanvl(df.a, df.b)).show() + +---+---+-----------+-----------+ + | a| b|nanvl(a, b)|nanvl(a, b)| + +---+---+-----------+-----------+ + |1.0|NaN| 1.0| 1.0| + |NaN|2.0| 2.0| 2.0| + +---+---+-----------+-----------+ """ return _invoke_function_over_columns("nanvl", col1, col2) @@ -7079,7 +7121,7 @@ def percentile( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str input column. + col : :class:`~pyspark.sql.Column` or column name percentage : :class:`~pyspark.sql.Column`, float, list of floats or tuple of floats percentage in decimal (must be between 0.0 and 1.0). frequency : :class:`~pyspark.sql.Column` or int is a positive numeric literal which @@ -7092,28 +7134,25 @@ def percentile( Examples -------- - >>> key = (col("id") % 3).alias("key") - >>> value = (randn(42) + key * 10).alias("value") + >>> from pyspark.sql import functions as sf + >>> key = (sf.col("id") % 3).alias("key") + >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select( - ... percentile("value", [0.25, 0.5, 0.75], lit(1)).alias("quantiles") - ... ).show() - +--------------------+ - | quantiles| - +--------------------+ - |[0.74419914941216...| - +--------------------+ + >>> df.select(sf.percentile("value", [0.25, 0.5, 0.75], sf.lit(1))).show(truncate=False) + +--------------------------------------------------------+ + |percentile(value, array(0.25, 0.5, 0.75), 1) | + +--------------------------------------------------------+ + |[0.7441991494121..., 9.9900713756..., 19.33740203080...]| + +--------------------------------------------------------+ - >>> df.groupBy("key").agg( - ... percentile("value", 0.5, lit(1)).alias("median") - ... ).show() - +---+--------------------+ - |key| median| - +---+--------------------+ - | 0|-0.03449962216667901| - | 1| 9.990389751837329| - | 2| 19.967859769284075| - +---+--------------------+ + >>> df.groupBy("key").agg(sf.percentile("value", sf.lit(0.5), sf.lit(1))).show() + +---+-------------------------+ + |key|percentile(value, 0.5, 1)| + +---+-------------------------+ + | 0| -0.03449962216667901| + | 1| 9.990389751837329| + | 2| 19.967859769284075| + +---+-------------------------+ """ percentage = lit(list(percentage)) if isinstance(percentage, (list, tuple)) else lit(percentage) return _invoke_function_over_columns("percentile", col, percentage, lit(frequency)) @@ -7137,7 +7176,7 @@ def percentile_approx( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column. percentage : :class:`~pyspark.sql.Column`, float, list of floats or tuple of floats percentage in decimal (must be between 0.0 and 1.0). @@ -7156,22 +7195,25 @@ def percentile_approx( Examples -------- - >>> key = (col("id") % 3).alias("key") - >>> value = (randn(42) + key * 10).alias("value") + >>> from pyspark.sql import functions as sf + >>> key = (sf.col("id") % 3).alias("key") + >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select( - ... percentile_approx("value", [0.25, 0.5, 0.75], 1000000).alias("quantiles") - ... ).printSchema() - root - |-- quantiles: array (nullable = true) - | |-- element: double (containsNull = false) + >>> df.select(sf.percentile_approx("value", [0.25, 0.5, 0.75], 1000000)).show(truncate=False) + +----------------------------------------------------------+ + |percentile_approx(value, array(0.25, 0.5, 0.75), 1000000) | + +----------------------------------------------------------+ + |[0.7264430125286..., 9.98975299938..., 19.335304783039...]| + +----------------------------------------------------------+ - >>> df.groupBy("key").agg( - ... percentile_approx("value", 0.5, lit(1000000)).alias("median") - ... ).printSchema() - root - |-- key: long (nullable = true) - |-- median: double (nullable = true) + >>> df.groupBy("key").agg(sf.percentile_approx("value", sf.lit(0.5), sf.lit(1000000))).show() + +---+--------------------------------------+ + |key|percentile_approx(value, 0.5, 1000000)| + +---+--------------------------------------+ + | 0| -0.03519435193070...| + | 1| 9.990389751837...| + | 2| 19.967859769284...| + +---+--------------------------------------+ """ percentage = lit(list(percentage)) if isinstance(percentage, (list, tuple)) else lit(percentage) return _invoke_function_over_columns("percentile_approx", col, percentage, lit(accuracy)) @@ -7191,7 +7233,7 @@ def approx_percentile( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column. percentage : :class:`~pyspark.sql.Column`, float, list of floats or tuple of floats percentage in decimal (must be between 0.0 and 1.0). @@ -7210,23 +7252,25 @@ def approx_percentile( Examples -------- - >>> import pyspark.sql.functions as sf + >>> from pyspark.sql import functions as sf >>> key = (sf.col("id") % 3).alias("key") >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select( - ... sf.approx_percentile("value", [0.25, 0.5, 0.75], 1000000) - ... ).printSchema() - root - |-- approx_percentile(value, array(0.25, 0.5, 0.75), 1000000): array (nullable = true) - | |-- element: double (containsNull = false) + >>> df.select(sf.approx_percentile("value", [0.25, 0.5, 0.75], 1000000)).show(truncate=False) + +----------------------------------------------------------+ + |approx_percentile(value, array(0.25, 0.5, 0.75), 1000000) | + +----------------------------------------------------------+ + |[0.7264430125286507, 9.98975299938167, 19.335304783039014]| + +----------------------------------------------------------+ - >>> df.groupBy("key").agg( - ... sf.approx_percentile("value", 0.5, sf.lit(1000000)) - ... ).printSchema() - root - |-- key: long (nullable = true) - |-- approx_percentile(value, 0.5, 1000000): double (nullable = true) + >>> df.groupBy("key").agg(sf.approx_percentile("value", sf.lit(0.5), sf.lit(1000000))).show() + +---+--------------------------------------+ + |key|approx_percentile(value, 0.5, 1000000)| + +---+--------------------------------------+ + | 0| -0.03519435193070876| + | 1| 9.990389751837329| + | 2| 19.967859769284075| + +---+--------------------------------------+ """ percentage = lit(list(percentage)) if isinstance(percentage, (list, tuple)) else lit(percentage) return _invoke_function_over_columns("approx_percentile", col, percentage, lit(accuracy)) @@ -7261,22 +7305,22 @@ def rand(seed: Optional[int] = None) -> Column: Example 1: Generate a random column without a seed >>> from pyspark.sql import functions as sf - >>> spark.range(0, 2, 1, 1).withColumn('rand', sf.rand()).show() # doctest: +SKIP - +---+-------------------+ - | id| rand| - +---+-------------------+ - | 0|0.14879325244215424| - | 1| 0.4640631044275454| - +---+-------------------+ + >>> spark.range(0, 2, 1, 1).select("*", sf.rand()).show() # doctest: +SKIP + +---+-------------------------+ + | id|rand(-158884697681280011)| + +---+-------------------------+ + | 0| 0.9253464547887...| + | 1| 0.6533254118758...| + +---+-------------------------+ Example 2: Generate a random column with a specific seed - >>> spark.range(0, 2, 1, 1).withColumn('rand', sf.rand(seed=42) * 3).show() + >>> spark.range(0, 2, 1, 1).select("*", sf.rand(seed=42)).show() +---+------------------+ - | id| rand| + | id| rand(42)| +---+------------------+ - | 0|1.8575681106759028| - | 1|1.5288056527339444| + | 0| 0.619189370225...| + | 1|0.5096018842446...| +---+------------------+ """ if seed is not None: @@ -7314,22 +7358,22 @@ def randn(seed: Optional[int] = None) -> Column: Example 1: Generate a random column without a seed >>> from pyspark.sql import functions as sf - >>> spark.range(0, 2, 1, 1).withColumn('randn', sf.randn()).show() # doctest: +SKIP - +---+--------------------+ - | id| randn| - +---+--------------------+ - | 0|-0.45011372342934214| - | 1| 0.6567304165329736| - +---+--------------------+ + >>> spark.range(0, 2, 1, 1).select("*", sf.randn()).show() # doctest: +SKIP + +---+--------------------------+ + | id|randn(3968742514375399317)| + +---+--------------------------+ + | 0| -0.47968645355788...| + | 1| -0.4950952457305...| + +---+--------------------------+ Example 2: Generate a random column with a specific seed - >>> spark.range(0, 2, 1, 1).withColumn('randn', sf.randn(seed=42)).show() + >>> spark.range(0, 2, 1, 1).select("*", sf.randn(seed=42)).show() +---+------------------+ - | id| randn| + | id| randn(42)| +---+------------------+ - | 0| 2.384479054241165| - | 1|0.1920934041293524| + | 0| 2.384479054241...| + | 1|0.1920934041293...| +---+------------------+ """ if seed is not None: @@ -7351,7 +7395,7 @@ def round(col: "ColumnOrName", scale: Optional[Union[Column, int]] = None) -> Co Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The target column or column name to compute the round on. scale : :class:`~pyspark.sql.Column` or int, optional An optional parameter to control the rounding behavior. @@ -7407,7 +7451,7 @@ def bround(col: "ColumnOrName", scale: Optional[Union[Column, int]] = None) -> C Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name The target column or column name to compute the round on. scale : :class:`~pyspark.sql.Column` or int, optional An optional parameter to control the rounding behavior. @@ -7477,7 +7521,7 @@ def shiftleft(col: "ColumnOrName", numBits: int) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column of values to shift. numBits : int number of bits to shift. @@ -7489,8 +7533,16 @@ def shiftleft(col: "ColumnOrName", numBits: int) -> Column: Examples -------- - >>> spark.createDataFrame([(21,)], ['a']).select(shiftleft('a', 1).alias('r')).collect() - [Row(r=42)] + >>> import pyspark.sql.functions as sf + >>> spark.range(4).select("*", sf.shiftleft('id', 1)).show() + +---+----------------+ + | id|shiftleft(id, 1)| + +---+----------------+ + | 0| 0| + | 1| 2| + | 2| 4| + | 3| 6| + +---+----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -7524,7 +7576,7 @@ def shiftright(col: "ColumnOrName", numBits: int) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column of values to shift. numBits : int number of bits to shift. @@ -7536,8 +7588,16 @@ def shiftright(col: "ColumnOrName", numBits: int) -> Column: Examples -------- - >>> spark.createDataFrame([(42,)], ['a']).select(shiftright('a', 1).alias('r')).collect() - [Row(r=21)] + >>> import pyspark.sql.functions as sf + >>> spark.range(4).select("*", sf.shiftright('id', 1)).show() + +---+-----------------+ + | id|shiftright(id, 1)| + +---+-----------------+ + | 0| 0| + | 1| 0| + | 2| 1| + | 3| 1| + +---+-----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -7571,7 +7631,7 @@ def shiftrightunsigned(col: "ColumnOrName", numBits: int) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column of values to shift. numBits : int number of bits to shift. @@ -7583,9 +7643,16 @@ def shiftrightunsigned(col: "ColumnOrName", numBits: int) -> Column: Examples -------- - >>> df = spark.createDataFrame([(-42,)], ['a']) - >>> df.select(shiftrightunsigned('a', 1).alias('r')).collect() - [Row(r=9223372036854775787)] + >>> import pyspark.sql.functions as sf + >>> spark.range(4).select("*", sf.shiftrightunsigned(sf.col('id') - 2, 1)).show() + +---+-------------------------------+ + | id|shiftrightunsigned((id - 2), 1)| + +---+-------------------------------+ + | 0| 9223372036854775807| + | 1| 9223372036854775807| + | 2| 0| + | 3| 0| + +---+-------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -7612,9 +7679,22 @@ def spark_partition_id() -> Column: Examples -------- - >>> df = spark.range(2) - >>> df.repartition(1).select(spark_partition_id().alias("pid")).collect() - [Row(pid=0), Row(pid=0)] + >>> import pyspark.sql.functions as sf + >>> spark.range(10, numPartitions=5).select("*", sf.spark_partition_id()).show() + +---+--------------------+ + | id|SPARK_PARTITION_ID()| + +---+--------------------+ + | 0| 0| + | 1| 0| + | 2| 1| + | 3| 1| + | 4| 2| + | 5| 2| + | 6| 3| + | 7| 3| + | 8| 4| + | 9| 4| + +---+--------------------+ """ return _invoke_function("spark_partition_id") @@ -7630,7 +7710,7 @@ def expr(str: str) -> Column: Parameters ---------- - str : str + str : expression string expression defined in string. Returns @@ -7640,8 +7720,9 @@ def expr(str: str) -> Column: Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([["Alice"], ["Bob"]], ["name"]) - >>> df.select("name", expr("length(name)")).show() + >>> df.select("*", sf.expr("length(name)")).show() +-----+------------+ | name|length(name)| +-----+------------+ From b61411d9eba4b9a96e4dc29017627d97ec73db3e Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 19 Nov 2024 08:41:48 +0800 Subject: [PATCH 07/58] [SPARK-50328][INFRA] Add a separate docker file for SparkR ### What changes were proposed in this pull request? Add a separate docker file for SparkR ### Why are the changes needed? For env isolation ### Does this PR introduce _any_ user-facing change? No, infra-only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #48859 from zhengruifeng/infra_image_r. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 25 +++++- .../workflows/build_infra_images_cache.yml | 14 ++++ dev/spark-test-image/sparkr/Dockerfile | 77 +++++++++++++++++++ 3 files changed, 115 insertions(+), 1 deletion(-) create mode 100644 dev/spark-test-image/sparkr/Dockerfile diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 4a3707404bccf..3117872e21680 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -62,6 +62,8 @@ jobs: image_docs_url_link: ${{ steps.infra-image-link.outputs.image_docs_url_link }} image_lint_url: ${{ steps.infra-image-lint-outputs.outputs.image_lint_url }} image_lint_url_link: ${{ steps.infra-image-link.outputs.image_lint_url_link }} + image_sparkr_url: ${{ steps.infra-image-sparkr-outputs.outputs.image_sparkr_url }} + image_sparkr_url_link: ${{ steps.infra-image-link.outputs.image_sparkr_url_link }} steps: - name: Checkout Spark repository uses: actions/checkout@v4 @@ -154,6 +156,14 @@ jobs: IMG_NAME="apache-spark-ci-image-lint:${{ inputs.branch }}-${{ github.run_id }}" IMG_URL="ghcr.io/$REPO_OWNER/$IMG_NAME" echo "image_lint_url=$IMG_URL" >> $GITHUB_OUTPUT + - name: Generate infra image URL (SparkR) + id: infra-image-sparkr-outputs + run: | + # Convert to lowercase to meet Docker repo name requirement + REPO_OWNER=$(echo "${{ github.repository_owner }}" | tr '[:upper:]' '[:lower:]') + IMG_NAME="apache-spark-ci-image-sparkr:${{ inputs.branch }}-${{ github.run_id }}" + IMG_URL="ghcr.io/$REPO_OWNER/$IMG_NAME" + echo "image_sparkr_url=$IMG_URL" >> $GITHUB_OUTPUT - name: Link the docker images id: infra-image-link run: | @@ -162,9 +172,11 @@ jobs: if [[ "${{ inputs.branch }}" == 'branch-3.5' ]]; then echo "image_docs_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT echo "image_lint_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT + echo "image_sparkr_url_link=${{ steps.infra-image-outputs.outputs.image_url }}" >> $GITHUB_OUTPUT else echo "image_docs_url_link=${{ steps.infra-image-docs-outputs.outputs.image_docs_url }}" >> $GITHUB_OUTPUT echo "image_lint_url_link=${{ steps.infra-image-lint-outputs.outputs.image_lint_url }}" >> $GITHUB_OUTPUT + echo "image_sparkr_url_link=${{ steps.infra-image-sparkr-outputs.outputs.image_sparkr_url }}" >> $GITHUB_OUTPUT fi # Build: build Spark and run the tests for specified modules. @@ -405,6 +417,17 @@ jobs: ${{ needs.precondition.outputs.image_lint_url }} # Use the infra image cache to speed up cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-lint-cache:${{ inputs.branch }} + - name: Build and push (SparkR) + if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' + id: docker_build_sparkr + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/sparkr/ + push: true + tags: | + ${{ needs.precondition.outputs.image_sparkr_url }} + # Use the infra image cache to speed up + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ inputs.branch }} pyspark: @@ -564,7 +587,7 @@ jobs: runs-on: ubuntu-latest timeout-minutes: 180 container: - image: ${{ needs.precondition.outputs.image_url }} + image: ${{ needs.precondition.outputs.image_sparkr_url_link }} env: HADOOP_PROFILE: ${{ inputs.hadoop }} HIVE_PROFILE: hive2.3 diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index b82d0633b0cee..a6beacedeebd4 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -29,6 +29,7 @@ on: - 'dev/infra/Dockerfile' - 'dev/spark-test-image/docs/Dockerfile' - 'dev/spark-test-image/lint/Dockerfile' + - 'dev/spark-test-image/sparkr/Dockerfile' - '.github/workflows/build_infra_images_cache.yml' # Create infra image when cutting down branches/tags create: @@ -88,3 +89,16 @@ jobs: - name: Image digest (Linter) if: hashFiles('dev/spark-test-image/lint/Dockerfile') != '' run: echo ${{ steps.docker_build_lint.outputs.digest }} + - name: Build and push (SparkR) + if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' + id: docker_build_sparkr + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/sparkr/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-sparkr-cache:${{ github.ref_name }},mode=max + - name: Image digest (SparkR) + if: hashFiles('dev/spark-test-image/sparkr/Dockerfile') != '' + run: echo ${{ steps.docker_build_sparkr.outputs.digest }} diff --git a/dev/spark-test-image/sparkr/Dockerfile b/dev/spark-test-image/sparkr/Dockerfile new file mode 100644 index 0000000000000..43260c714a550 --- /dev/null +++ b/dev/spark-test-image/sparkr/Dockerfile @@ -0,0 +1,77 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image for SparkR" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE 20241114 + +ENV DEBIAN_FRONTEND noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + pandoc \ + pkg-config \ + qpdf \ + r-base \ + software-properties-common \ + wget \ + zlib1g-dev \ + && rm -rf /var/lib/apt/lists/* + +RUN echo 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' >> /etc/apt/sources.list +RUN gpg --keyserver hkps://keyserver.ubuntu.com --recv-key E298A3A825C0D65DFD57CBB651716619E084DAB9 +RUN gpg -a --export E084DAB9 | apt-key add - +RUN add-apt-repository 'deb https://cloud.r-project.org/bin/linux/ubuntu jammy-cran40/' + +# See more in SPARK-39959, roxygen2 < 7.2.1 +RUN Rscript -e "install.packages(c('devtools', 'knitr', 'markdown', \ + 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', \ + 'ggplot2', 'mvtnorm', 'statmod', 'xml2'), repos='https://cloud.r-project.org/')" && \ + Rscript -e "devtools::install_version('roxygen2', version='7.2.0', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('lintr', version='2.0.1', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('pkgdown', version='2.0.1', repos='https://cloud.r-project.org')" && \ + Rscript -e "devtools::install_version('preferably', version='0.4', repos='https://cloud.r-project.org')" + +# See more in SPARK-39735 +ENV R_LIBS_SITE "/usr/local/lib/R/site-library:${R_LIBS_SITE}:/usr/lib/R/library" From e1477a34d7457a1f31164e4e70e00c9912a8de4b Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Tue, 19 Nov 2024 09:41:42 +0800 Subject: [PATCH 08/58] [SPARK-50298][PYTHON][CONNECT] Implement verifySchema parameter of createDataFrame in Spark Connect ### What changes were proposed in this pull request? The PR targets at Spark Connect only. Spark Classic has been handled in https://github.com/apache/spark/pull/48677. `verifySchema` parameter of createDataFrame on Spark Classic decides whether to verify data types of every row against schema. Now it's not supported on Spark Connect. The PR proposes to support `verifySchema` on Spark Connect. By default, `verifySchema` parameter is `pyspark._NoValue`, if not provided, createDataFrame with - `pyarrow.Table`, **verifySchema = False** - `pandas.DataFrame` with Arrow optimization, **verifySchema = spark.sql.execution.pandas.convertToArrowArraySafely** - regular Python instances, **verifySchema = True** The schema enforcement of numpy ndarray input is unexpected and will be resolved as a follow-up, https://issues.apache.org/jira/browse/SPARK-50323. ### Why are the changes needed? Parity with Spark Classic. ### Does this PR introduce _any_ user-facing change? Yes, `verifySchema` parameter of createDataFrame is supported in Spark Connect. ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48841 from xinrong-meng/verifySchemaConnect. Authored-by: Xinrong Meng Signed-off-by: Xinrong Meng --- python/pyspark/sql/connect/conversion.py | 6 +-- python/pyspark/sql/connect/session.py | 30 +++++++++--- .../sql/tests/connect/test_parity_arrow.py | 3 +- python/pyspark/sql/tests/test_arrow.py | 46 +++++++++++-------- 4 files changed, 54 insertions(+), 31 deletions(-) diff --git a/python/pyspark/sql/connect/conversion.py b/python/pyspark/sql/connect/conversion.py index d803f37c5b9f1..f689c439f5f68 100644 --- a/python/pyspark/sql/connect/conversion.py +++ b/python/pyspark/sql/connect/conversion.py @@ -322,7 +322,7 @@ def convert_other(value: Any) -> Any: return lambda value: value @staticmethod - def convert(data: Sequence[Any], schema: StructType) -> "pa.Table": + def convert(data: Sequence[Any], schema: StructType, verifySchema: bool = False) -> "pa.Table": assert isinstance(data, list) and len(data) > 0 assert schema is not None and isinstance(schema, StructType) @@ -372,8 +372,8 @@ def convert(data: Sequence[Any], schema: StructType) -> "pa.Table": ] ) ) - - return pa.Table.from_arrays(pylist, schema=pa_schema) + table = pa.Table.from_arrays(pylist, schema=pa_schema) + return table.cast(pa_schema, safe=verifySchema) class ArrowTableToRowsConversion: diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 83b0496a84274..e7292bf8804f8 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -50,6 +50,7 @@ ) import urllib +from pyspark._globals import _NoValue, _NoValueType from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.dataframe import DataFrame as ParentDataFrame from pyspark.sql.connect.logging import logger @@ -449,7 +450,7 @@ def createDataFrame( data: Union["pd.DataFrame", "np.ndarray", "pa.Table", Iterable[Any]], schema: Optional[Union[AtomicType, StructType, str, List[str], Tuple[str, ...]]] = None, samplingRatio: Optional[float] = None, - verifySchema: Optional[bool] = None, + verifySchema: Union[_NoValueType, bool] = _NoValue, ) -> "ParentDataFrame": assert data is not None if isinstance(data, DataFrame): @@ -461,9 +462,6 @@ def createDataFrame( if samplingRatio is not None: warnings.warn("'samplingRatio' is ignored. It is not supported with Spark Connect.") - if verifySchema is not None: - warnings.warn("'verifySchema' is ignored. It is not supported with Spark Connect.") - _schema: Optional[Union[AtomicType, StructType]] = None _cols: Optional[List[str]] = None _num_cols: Optional[int] = None @@ -576,7 +574,10 @@ def createDataFrame( "spark.sql.session.timeZone", "spark.sql.execution.pandas.convertToArrowArraySafely" ) - ser = ArrowStreamPandasSerializer(cast(str, timezone), safecheck == "true") + if verifySchema is _NoValue: + verifySchema = safecheck == "true" + + ser = ArrowStreamPandasSerializer(cast(str, timezone), verifySchema) _table = pa.Table.from_batches( [ @@ -596,6 +597,9 @@ def createDataFrame( ).cast(arrow_schema) elif isinstance(data, pa.Table): + if verifySchema is _NoValue: + verifySchema = False + prefer_timestamp_ntz = is_timestamp_ntz_preferred() (timezone,) = self._client.get_configs("spark.sql.session.timeZone") @@ -613,7 +617,10 @@ def createDataFrame( _table = ( _check_arrow_table_timestamps_localize(data, schema, True, timezone) - .cast(to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True)) + .cast( + to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True), + safe=verifySchema, + ) .rename_columns(schema.names) ) @@ -652,6 +659,12 @@ def createDataFrame( # The _table should already have the proper column names. _cols = None + if verifySchema is not _NoValue: + warnings.warn( + "'verifySchema' is ignored. It is not supported" + " with np.ndarray input on Spark Connect." + ) + else: _data = list(data) @@ -683,12 +696,15 @@ def createDataFrame( errorClass="CANNOT_DETERMINE_TYPE", messageParameters={} ) + if verifySchema is _NoValue: + verifySchema = True + from pyspark.sql.connect.conversion import LocalDataToArrowConversion # Spark Connect will try its best to build the Arrow table with the # inferred schema in the client side, and then rename the columns and # cast the datatypes in the server side. - _table = LocalDataToArrowConversion.convert(_data, _schema) + _table = LocalDataToArrowConversion.convert(_data, _schema, cast(bool, verifySchema)) # TODO: Beside the validation on number of columns, we should also check # whether the Arrow Schema is compatible with the user provided Schema. diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow.py b/python/pyspark/sql/tests/connect/test_parity_arrow.py index d47a367a5460a..99d03ad1a4409 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow.py +++ b/python/pyspark/sql/tests/connect/test_parity_arrow.py @@ -137,9 +137,8 @@ def test_toPandas_udt(self): def test_create_dataframe_namedtuples(self): self.check_create_dataframe_namedtuples(True) - @unittest.skip("Spark Connect does not support verifySchema.") def test_createDataFrame_verifySchema(self): - super().test_createDataFrame_verifySchema() + self.check_createDataFrame_verifySchema(True) if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 19d0db9894317..99149d1a23d3e 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -533,6 +533,11 @@ def test_createDataFrame_arrow_pandas(self): self.assertEqual(df_arrow.collect(), df_pandas.collect()) def test_createDataFrame_verifySchema(self): + for arrow_enabled in [True, False]: + with self.subTest(arrow_enabled=arrow_enabled): + self.check_createDataFrame_verifySchema(arrow_enabled) + + def check_createDataFrame_verifySchema(self, arrow_enabled): data = {"id": [1, 2, 3], "value": [100000000000, 200000000000, 300000000000]} # data.value should fail schema validation when verifySchema is True schema = StructType( @@ -547,29 +552,32 @@ def test_createDataFrame_verifySchema(self): table = pa.table(data) df = self.spark.createDataFrame(table, schema=schema) self.assertEqual(df.collect(), expected) - with self.assertRaises(Exception): self.spark.createDataFrame(table, schema=schema, verifySchema=True) - # pandas DataFrame with Arrow optimization - pdf = pd.DataFrame(data) - df = self.spark.createDataFrame(pdf, schema=schema) - # verifySchema defaults to `spark.sql.execution.pandas.convertToArrowArraySafely`, - # which is false by default - self.assertEqual(df.collect(), expected) - with self.assertRaises(Exception): - with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": True}): + if arrow_enabled: + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": True}): + # pandas DataFrame with Arrow optimization + pdf = pd.DataFrame(data) df = self.spark.createDataFrame(pdf, schema=schema) - with self.assertRaises(Exception): - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=True) - - # pandas DataFrame without Arrow optimization - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): - pdf = pd.DataFrame(data) - with self.assertRaises(Exception): - df = self.spark.createDataFrame(pdf, schema=schema) # verifySchema defaults to True - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=False) - self.assertEqual(df.collect(), expected) + # verifySchema defaults to `spark.sql.execution.pandas.convertToArrowArraySafely`, + # which is false by default + self.assertEqual(df.collect(), expected) + with self.assertRaises(Exception): + with self.sql_conf( + {"spark.sql.execution.pandas.convertToArrowArraySafely": True} + ): + df = self.spark.createDataFrame(pdf, schema=schema) + with self.assertRaises(Exception): + df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=True) + else: + # pandas DataFrame without Arrow optimization + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + pdf = pd.DataFrame(data) + with self.assertRaises(Exception): + self.spark.createDataFrame(pdf, schema=schema) # verifySchema defaults to True + df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=False) + self.assertEqual(df.collect(), expected) def _createDataFrame_toggle(self, data, schema=None): with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): From 6d4798103502bafd1888a7cb3313f0534c34b559 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 19 Nov 2024 10:44:40 +0800 Subject: [PATCH 09/58] [SPARK-50331][INFRA] Add a daily test for PySpark on MacOS ### What changes were proposed in this pull request? This pr aims to add a daily test for PySpark on macos-15 ### Why are the changes needed? Check PySpark on macos-15. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #48868 from LuciferYang/python-macos. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .github/workflows/build_python_3.11_macos.yml | 32 ++++ .github/workflows/python_macos_test.yml | 165 ++++++++++++++++++ 2 files changed, 197 insertions(+) create mode 100644 .github/workflows/build_python_3.11_macos.yml create mode 100644 .github/workflows/python_macos_test.yml diff --git a/.github/workflows/build_python_3.11_macos.yml b/.github/workflows/build_python_3.11_macos.yml new file mode 100644 index 0000000000000..4caae55b5fea8 --- /dev/null +++ b/.github/workflows/build_python_3.11_macos.yml @@ -0,0 +1,32 @@ +# +# 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. +# + +name: "Build / Python-only (master, Python 3.11, MacOS)" + +on: + schedule: + - cron: '0 21 * * *' + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/python_macos_test.yml + if: github.repository == 'apache/spark' diff --git a/.github/workflows/python_macos_test.yml b/.github/workflows/python_macos_test.yml new file mode 100644 index 0000000000000..32a8e21801db3 --- /dev/null +++ b/.github/workflows/python_macos_test.yml @@ -0,0 +1,165 @@ +# +# 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. +# + +name: Build and test PySpark on macOS + +on: + workflow_call: + inputs: + java: + required: false + type: string + default: 17 + python: + required: false + type: string + default: 3.11 + branch: + description: Branch to run the build against + required: false + type: string + default: master + hadoop: + description: Hadoop version to run with. HADOOP_PROFILE environment variable should accept it. + required: false + type: string + default: hadoop3 + envs: + description: Additional environment variables to set when running the tests. Should be in JSON format. + required: false + type: string + default: '{}' +jobs: + build: + name: "PySpark test on macos: ${{ matrix.modules }}" + runs-on: macos-15 + strategy: + fail-fast: false + matrix: + java: + - ${{ inputs.java }} + python: + - ${{inputs.python}} + modules: + - >- + pyspark-sql, pyspark-resource, pyspark-testing + - >- + pyspark-core, pyspark-errors, pyspark-streaming + - >- + pyspark-mllib, pyspark-ml, pyspark-ml-connect + - >- + pyspark-connect + - >- + pyspark-pandas + - >- + pyspark-pandas-slow + - >- + pyspark-pandas-connect-part0 + - >- + pyspark-pandas-connect-part1 + - >- + pyspark-pandas-connect-part2 + - >- + pyspark-pandas-connect-part3 + env: + MODULES_TO_TEST: ${{ matrix.modules }} + PYTHON_TO_TEST: python${{inputs.python}} + HADOOP_PROFILE: ${{ inputs.hadoop }} + HIVE_PROFILE: hive2.3 + # GitHub Actions' default miniconda to use in pip packaging test. + CONDA_PREFIX: /usr/share/miniconda + GITHUB_PREV_SHA: ${{ github.event.before }} + SPARK_LOCAL_IP: localhost + SKIP_UNIDOC: true + SKIP_MIMA: true + SKIP_PACKAGING: true + METASPACE_SIZE: 1g + BRANCH: ${{ inputs.branch }} + steps: + - name: Checkout Spark repository + uses: actions/checkout@v4 + # In order to fetch changed files + with: + fetch-depth: 0 + repository: apache/spark + ref: ${{ inputs.branch }} + - name: Sync the current branch with the latest in Apache Spark + if: github.repository != 'apache/spark' + run: | + echo "APACHE_SPARK_REF=$(git rev-parse HEAD)" >> $GITHUB_ENV + git fetch https://github.com/$GITHUB_REPOSITORY.git ${GITHUB_REF#refs/heads/} + git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' merge --no-commit --progress --squash FETCH_HEAD + git -c user.name='Apache Spark Test Account' -c user.email='sparktestacc@gmail.com' commit -m "Merged commit" --allow-empty + # Cache local repositories. Note that GitHub Actions cache has a 10G limit. + - name: Cache SBT and Maven + uses: actions/cache@v4 + with: + path: | + build/apache-maven-* + build/*.jar + ~/.sbt + key: build-${{ hashFiles('**/pom.xml', 'project/build.properties', 'build/mvn', 'build/sbt', 'build/sbt-launch-lib.bash', 'build/spark-build-info') }} + restore-keys: | + build- + - name: Cache Coursier local repository + uses: actions/cache@v4 + with: + path: ~/.cache/coursier + key: pyspark-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} + restore-keys: | + pyspark-coursier- + - name: Install Java ${{ matrix.java }} + uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: ${{ matrix.java }} + - name: Install Python packages (Python ${{matrix.python}}) + run: | + python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' + python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' + python${{matrix.python}} -m pip install py-cpuinfo && \ + python${{matrix.python}} -m pip install numpy 'pyarrow>=15.0.0' 'six==1.16.0' 'pandas==2.2.3' scipy 'plotly>=4.8' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ + python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ + python${{matrix.python}} -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ + python${{matrix.python}} -m pip install deepspeed torcheval && \ + python${{matrix.python}} -m pip cache purge && \ + python${{matrix.python}} -m pip list + # Run the tests. + - name: Run tests + env: ${{ fromJSON(inputs.envs) }} + run: | + if [[ "$MODULES_TO_TEST" == *"pyspark-errors"* ]]; then + export SKIP_PACKAGING=false + echo "Python Packaging Tests Enabled!" + fi + ./dev/run-tests --parallelism 1 --modules "$MODULES_TO_TEST" --python-executables "$PYTHON_TO_TEST" + - name: Upload test results to report + env: ${{ fromJSON(inputs.envs) }} + if: always() + uses: actions/upload-artifact@v4 + with: + name: test-results-${{ matrix.modules }}--${{ matrix.java }}-${{ inputs.hadoop }}-hive2.3-${{ env.PYTHON_TO_TEST }} + path: "**/target/test-reports/*.xml" + - name: Upload unit tests log files + env: ${{ fromJSON(inputs.envs) }} + if: ${{ !success() }} + uses: actions/upload-artifact@v4 + with: + name: unit-tests-log-${{ matrix.modules }}--${{ matrix.java }}-${{ inputs.hadoop }}-hive2.3-${{ env.PYTHON_TO_TEST }} + path: "**/target/unit-tests.log" From 5a57efdcee9e6569d8de4272bda258788cf349e3 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 19 Nov 2024 11:06:38 +0800 Subject: [PATCH 10/58] [SPARK-50313][SQL][TESTS][FOLLOWUP] Restore some tests in *SQLQueryTestSuite for ANSI mode ### What changes were proposed in this pull request? This PR makes a follow-up for SPARK-50313, to restore some tests in *SQLQueryTestSuite for ANSI mode. These tests are originally defined in ansi/ folder, in other words, they are not `--IMPORT`-ed, so we need to move them to the default folder where now defaults to ANSI after SPARK-50313. ### Why are the changes needed? Improve ANSI test coverage. ### Does this PR introduce _any_ user-facing change? NO ### How was this patch tested? *SQLQueryTestSuite ### Was this patch authored or co-authored using generative AI tooling? NO Closes #48872 from yaooqinn/SPARK-50313-F. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../conditional-functions.sql.out | 142 ++++++ .../double-quoted-identifiers-enabled.sql.out | 261 ++++++---- .../group-by-all-mosha.sql.out | 6 +- .../keywords-enforced.sql.out | 16 + .../decimalArithmeticOperations.sql.out | 277 +++++++++++ ....out => double-quoted-identifiers.sql.out} | 0 .../sql-tests/analyzer-results/union.sql.out | 13 +- .../inputs/conditional-functions.sql | 38 ++ .../double-quoted-identifiers-enabled.sql | 0 .../sql-tests/inputs/group-by-all-mosha.sql | 2 +- .../sql-tests/inputs/keywords-enforced.sql | 2 + .../inputs/nonansi/conditional-functions.sql | 39 +- .../nonansi/decimalArithmeticOperations.sql | 42 +- .../double-quoted-identifiers-disabled.sql | 2 - .../nonansi/double-quoted-identifiers.sql | 1 + .../sql-tests/inputs/nonansi/keywords.sql | 1 - .../test/resources/sql-tests/inputs/union.sql | 2 +- .../results/conditional-functions.sql.out | 202 ++++++++ .../double-quoted-identifiers-enabled.sql.out | 238 +++++---- .../results/group-by-all-mosha.sql.out | 24 +- .../results/keywords-enforced.sql.out | 453 ++++++++++++++++++ .../decimalArithmeticOperations.sql.out | 304 +++++++++++- ....out => double-quoted-identifiers.sql.out} | 0 .../resources/sql-tests/results/union.sql.out | 6 +- 24 files changed, 1744 insertions(+), 327 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/conditional-functions.sql.out rename sql/core/src/test/resources/sql-tests/analyzer-results/{nonansi => }/double-quoted-identifiers-enabled.sql.out (56%) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/keywords-enforced.sql.out rename sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/{double-quoted-identifiers-disabled.sql.out => double-quoted-identifiers.sql.out} (100%) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/conditional-functions.sql rename sql/core/src/test/resources/sql-tests/inputs/{nonansi => }/double-quoted-identifiers-enabled.sql (100%) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/keywords-enforced.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql create mode 100644 sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/conditional-functions.sql.out rename sql/core/src/test/resources/sql-tests/results/{nonansi => }/double-quoted-identifiers-enabled.sql.out (66%) create mode 100644 sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out rename sql/core/src/test/resources/sql-tests/results/nonansi/{double-quoted-identifiers-disabled.sql.out => double-quoted-identifiers.sql.out} (100%) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/conditional-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/conditional-functions.sql.out new file mode 100644 index 0000000000000..5effa73c413a6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/conditional-functions.sql.out @@ -0,0 +1,142 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2) +-- !query analysis +CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`conditional_t`, ErrorIfExists, [c1, c2] + +- Project [c1#x, c2#x] + +- SubqueryAlias t + +- LocalRelation [c1#x, c2#x] + + +-- !query +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t +-- !query analysis +Project [nanvl(cast(c2#x as double), ((c1#x / cast(c2#x as double)) + (c1#x / cast(c2#x as double)))) AS nanvl(c2, ((c1 / c2) + (c1 / c2)))#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT nanvl(c2, 1/0) FROM conditional_t +-- !query analysis +Project [nanvl(cast(c2#x as double), (cast(1 as double) / cast(0 as double))) AS nanvl(c2, (1 / 0))#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT nanvl(1-0, 1/0) FROM conditional_t +-- !query analysis +Project [nanvl(cast((1 - 0) as double), (cast(1 as double) / cast(0 as double))) AS nanvl((1 - 0), (1 / 0))#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t +-- !query analysis +Project [if ((c2#x >= 0)) cast((1 - 0) as double) else (cast(1 as double) / cast(0 as double)) AS (IF((c2 >= 0), (1 - 0), (1 / 0)))#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT if(1 == 1, 1, 1/0) +-- !query analysis +Project [if ((1 = 1)) cast(1 as double) else (cast(1 as double) / cast(0 as double)) AS (IF((1 = 1), 1, (1 / 0)))#x] ++- OneRowRelation + + +-- !query +SELECT if(1 != 1, 1/0, 1) +-- !query analysis +Project [if (NOT (1 = 1)) (cast(1 as double) / cast(0 as double)) else cast(1 as double) AS (IF((NOT (1 = 1)), (1 / 0), 1))#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(c2, 1/0) from conditional_t +-- !query analysis +Project [coalesce(cast(c2#x as double), (cast(1 as double) / cast(0 as double))) AS coalesce(c2, (1 / 0))#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT coalesce(1, 1/0) +-- !query analysis +Project [coalesce(cast(1 as double), (cast(1 as double) / cast(0 as double))) AS coalesce(1, (1 / 0))#x] ++- OneRowRelation + + +-- !query +SELECT coalesce(null, 1, 1/0) +-- !query analysis +Project [coalesce(cast(null as double), cast(1 as double), (cast(1 as double) / cast(0 as double))) AS coalesce(NULL, 1, (1 / 0))#x] ++- OneRowRelation + + +-- !query +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t +-- !query analysis +Project [CASE WHEN (c2#x >= 0) THEN cast(1 as double) ELSE (cast(1 as double) / cast(0 as double)) END AS CASE WHEN (c2 >= 0) THEN 1 ELSE (1 / 0) END#x] ++- SubqueryAlias spark_catalog.default.conditional_t + +- Relation spark_catalog.default.conditional_t[c1#x,c2#x] parquet + + +-- !query +SELECT case when 1 < 2 then 1 else 1/0 end +-- !query analysis +Project [CASE WHEN (1 < 2) THEN cast(1 as double) ELSE (cast(1 as double) / cast(0 as double)) END AS CASE WHEN (1 < 2) THEN 1 ELSE (1 / 0) END#x] ++- OneRowRelation + + +-- !query +SELECT case when 1 > 2 then 1/0 else 1 end +-- !query analysis +Project [CASE WHEN (1 > 2) THEN (cast(1 as double) / cast(0 as double)) ELSE cast(1 as double) END AS CASE WHEN (1 > 2) THEN (1 / 0) ELSE 1 END#x] ++- OneRowRelation + + +-- !query +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null) +-- !query analysis +Project [nullifzero(0) AS nullifzero(0)#x, nullifzero(cast(0 as tinyint)) AS nullifzero(CAST(0 AS TINYINT))#x, nullifzero(cast(0 as bigint)) AS nullifzero(CAST(0 AS BIGINT))#xL, nullifzero(0) AS nullifzero(0)#x, nullifzero(0.0) AS nullifzero(0.0)#x, nullifzero(1) AS nullifzero(1)#x, nullifzero(null) AS nullifzero(NULL)#x] ++- OneRowRelation + + +-- !query +SELECT nullifzero('abc') +-- !query analysis +Project [nullifzero(abc) AS nullifzero(abc)#x] ++- OneRowRelation + + +-- !query +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)) +-- !query analysis +Project [zeroifnull(null) AS zeroifnull(NULL)#x, zeroifnull(1) AS zeroifnull(1)#x, zeroifnull(cast(1 as tinyint)) AS zeroifnull(CAST(1 AS TINYINT))#x, zeroifnull(cast(1 as bigint)) AS zeroifnull(CAST(1 AS BIGINT))#xL] ++- OneRowRelation + + +-- !query +SELECT zeroifnull('abc') +-- !query analysis +Project [zeroifnull(abc) AS zeroifnull(abc)#xL] ++- OneRowRelation + + +-- !query +DROP TABLE conditional_t +-- !query analysis +DropTable false, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.conditional_t diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out similarity index 56% rename from sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out index a02bf525f947d..22dfeac5fd0b6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out @@ -2,27 +2,32 @@ -- !query SELECT 1 FROM "not_exist" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "\"not_exist\"" + } ] } -- !query USE SCHEMA "not_exist" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" + "schemaName" : "`spark_catalog`.`not_exist`" } } @@ -30,84 +35,122 @@ org.apache.spark.sql.catalyst.parser.ParseException -- !query ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "\"not_exist\"" + } ] } -- !query ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] } -- !query SELECT 1 AS "not_exist" FROM not_exist -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 38, + "fragment" : "not_exist" + } ] } -- !query SELECT 1 FROM not_exist AS X("hello") -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"hello\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] } -- !query SELECT "not_exist"() -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "\"not_exist\"()" + } ] } -- !query SELECT "not_exist".not_exist() -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "\"not_exist\".not_exist()" + } ] } @@ -269,29 +312,62 @@ org.apache.spark.sql.AnalysisException -- !query SELECT "hello" -- !query analysis -Project [hello AS hello#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "\"hello\"" + } ] +} -- !query CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 -- !query analysis -CreateViewCommand `v`, [(c1,Some(hello))], SELECT 1, false, false, LocalTempView, UNSUPPORTED, true - +- Project [1 AS 1#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} -- !query DROP VIEW v -- !query analysis -DropTempViewCommand v +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`v`" + } +} -- !query SELECT INTERVAL "1" YEAR -- !query analysis -Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] -+- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"1\"'", + "hint" : "" + } +} -- !query @@ -325,69 +401,54 @@ Project [INTERVAL '1' YEAR AS INTERVAL '1' YEAR#x] -- !query CREATE SCHEMA "myschema" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myschema\"'", - "hint" : "" - } -} +CreateNamespace false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [myschema] -- !query CREATE TEMPORARY VIEW "myview"("c1") AS WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myview\"'", - "hint" : "" - } -} +CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, UNSUPPORTED, true + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [1#x AS a#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [a#x] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [a#x], false -- !query SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"a2\"'", - "hint" : "" - } -} +Project [a1#x AS a2#x] ++- SubqueryAlias atab + +- Project [c1#x AS a1#x] + +- SubqueryAlias myview + +- View (`myview`, [c1#x]) + +- Project [cast(a#x as int) AS c1#x] + +- WithCTE + :- CTERelationDef xxxx, false + : +- SubqueryAlias v + : +- Project [1#x AS a#x] + : +- Project [1 AS 1#x] + : +- OneRowRelation + +- Project [a#x] + +- SubqueryAlias v + +- CTERelationRef xxxx, true, [a#x], false -- !query DROP TABLE "myview" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myview\"'", - "hint" : "" - } -} +DropTempViewCommand myview -- !query DROP SCHEMA "myschema" -- !query analysis -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myschema\"'", - "hint" : "" - } -} +DropNamespace false, false ++- ResolvedNamespace V2SessionCatalog(spark_catalog), [myschema] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out index b7dd089ba86a8..34d22ad6c6176 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/group-by-all-mosha.sql.out @@ -56,11 +56,11 @@ Sort [i#x ASC NULLS FIRST, i#x ASC NULLS FIRST, ci#xL ASC NULLS FIRST, ci#xL ASC -- !query -SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff +SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), octet_length(d), size(a) FROM stuff GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2 -- !query analysis -Sort [(i + 1)#x ASC NULLS FIRST, substring(s, 2, 3)#x ASC NULLS FIRST, extract(year FROM t)#x ASC NULLS FIRST, (d / 2)#x ASC NULLS FIRST, size(a)#x ASC NULLS FIRST, (f / i)#x ASC NULLS FIRST], true -+- Aggregate [(i#x + 1), (f#x / cast(i#x as decimal(10,0))), substring(s#x, 2, 3), extract(year, t#x), (cast(cast(d#x as bigint) as double) / cast(cast(2 as bigint) as double)), size(a#x, false)], [(i#x + 1) AS (i + 1)#x, (f#x / cast(i#x as decimal(10,0))) AS (f / i)#x, substring(s#x, 2, 3) AS substring(s, 2, 3)#x, extract(year, t#x) AS extract(year FROM t)#x, (cast(cast(d#x as bigint) as double) / cast(cast(2 as bigint) as double)) AS (d / 2)#x, size(a#x, false) AS size(a)#x] +Sort [(i + 1)#x ASC NULLS FIRST, substring(s, 2, 3)#x ASC NULLS FIRST, extract(year FROM t)#x ASC NULLS FIRST, octet_length(d)#x ASC NULLS FIRST, size(a)#x ASC NULLS FIRST, (f / i)#x ASC NULLS FIRST], true ++- Aggregate [(i#x + 1), (f#x / cast(i#x as decimal(10,0))), substring(s#x, 2, 3), extract(year, t#x), octet_length(d#x), size(a#x, false)], [(i#x + 1) AS (i + 1)#x, (f#x / cast(i#x as decimal(10,0))) AS (f / i)#x, substring(s#x, 2, 3) AS substring(s, 2, 3)#x, extract(year, t#x) AS extract(year FROM t)#x, octet_length(d#x) AS octet_length(d)#x, size(a#x, false) AS size(a)#x] +- SubqueryAlias stuff +- View (`stuff`, [i#x, f#x, s#x, t#x, d#x, a#x]) +- Project [cast(i#x as int) AS i#x, cast(f#x as decimal(6,4)) AS f#x, cast(s#x as string) AS s#x, cast(t#x as string) AS t#x, cast(d#x as string) AS d#x, cast(a#x as array) AS a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/keywords-enforced.sql.out new file mode 100644 index 0000000000000..a549a03316bce --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/keywords-enforced.sql.out @@ -0,0 +1,16 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * from SQL_KEYWORDS() +-- !query analysis +Project [keyword#x, reserved#x] ++- Generate sql_keywords(), false, [keyword#x, reserved#x] + +- OneRowRelation + + +-- !query +SELECT keyword from SQL_KEYWORDS() WHERE reserved +-- !query analysis +Project [keyword#x] ++- Filter reserved#x: boolean + +- Generate sql_keywords(), false, [keyword#x, reserved#x] + +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out index bcaa991ddae04..d75f4d41bd425 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/decimalArithmeticOperations.sql.out @@ -1,4 +1,45 @@ -- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b +-- !query analysis +CreateViewCommand `t`, SELECT 1.0 as a, 0.0 as b, false, false, LocalTempView, UNSUPPORTED, true + +- Project [1.0 AS a#x, 0.0 AS b#x] + +- OneRowRelation + + +-- !query +select a / b from t +-- !query analysis +Project [(a#x / b#x) AS (a / b)#x] ++- SubqueryAlias t + +- View (`t`, [a#x, b#x]) + +- Project [cast(a#x as decimal(2,1)) AS a#x, cast(b#x as decimal(1,1)) AS b#x] + +- Project [1.0 AS a#x, 0.0 AS b#x] + +- OneRowRelation + + +-- !query +select a % b from t +-- !query analysis +Project [(a#x % b#x) AS (a % b)#x] ++- SubqueryAlias t + +- View (`t`, [a#x, b#x]) + +- Project [cast(a#x as decimal(2,1)) AS a#x, cast(b#x as decimal(1,1)) AS b#x] + +- Project [1.0 AS a#x, 0.0 AS b#x] + +- OneRowRelation + + +-- !query +select pmod(a, b) from t +-- !query analysis +Project [pmod(a#x, b#x) AS pmod(a, b)#x] ++- SubqueryAlias t + +- View (`t`, [a#x, b#x]) + +- Project [cast(a#x as decimal(2,1)) AS a#x, cast(b#x as decimal(1,1)) AS b#x] + +- Project [1.0 AS a#x, 0.0 AS b#x] + +- OneRowRelation + + -- !query create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet -- !query analysis @@ -14,6 +55,15 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d +- LocalRelation [col1#x, col2#x, col3#x] +-- !query +select id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [id#x, (a#x + b#x) AS (a + b)#x, (a#x - b#x) AS (a - b)#x, (a#x * b#x) AS (a * b)#x, (a#x / b#x) AS (a / b)#x] + +- SubqueryAlias spark_catalog.default.decimals_test + +- Relation spark_catalog.default.decimals_test[id#x,a#x,b#x] parquet + + -- !query select id, a*10, b/10 from decimals_test order by id -- !query analysis @@ -58,6 +108,13 @@ Project [(10.300000000000000000 * 3.0000000000000000000) AS (10.3000000000000000 +- OneRowRelation +-- !query +select 2.35E10 * 1.0 +-- !query analysis +Project [(2.35E10 * cast(1.0 as double)) AS (2.35E10 * 1.0)#x] ++- OneRowRelation + + -- !query select (5e36BD + 0.1) + 5e36BD -- !query analysis @@ -86,6 +143,20 @@ Project [(100000000000000000000000000000000000 / 0.1) AS (1000000000000000000000 +- OneRowRelation +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query analysis +Project [(1234567890123456789000000000000 * 12345678901234567890000000) AS (1234567890123456789000000000000 * 12345678901234567890000000)#x] ++- OneRowRelation + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query analysis +Project [(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345) AS (12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345)#x] ++- OneRowRelation + + -- !query select 123456789123456789.1234567890 * 1.123456789123456789 -- !query analysis @@ -93,6 +164,212 @@ Project [(123456789123456789.1234567890 * 1.123456789123456789) AS (123456789123 +- OneRowRelation +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query analysis +Project [(12345678912345.123456789123 / 1.2345678E-8) AS (12345678912345.123456789123 / 1.2345678E-8)#x] ++- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1)) +-- !query analysis +Union false, false +:- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [(cast(10 as decimal(3,1)) + cast(90 as decimal(3,1))) AS (CAST(10 AS DECIMAL(3,1)) + CAST(90 AS DECIMAL(3,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1)) +-- !query analysis +Union false, false +:- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [(cast(10 as decimal(3,1)) - cast(-90 as decimal(3,1))) AS (CAST(10 AS DECIMAL(3,1)) - CAST(-90 AS DECIMAL(3,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1)) +-- !query analysis +Union false, false +:- Project [cast(CAST(20 AS DECIMAL(4,1))#x as decimal(7,2)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [(cast(10 as decimal(3,1)) * cast(10 as decimal(3,1))) AS (CAST(10 AS DECIMAL(3,1)) * CAST(10 AS DECIMAL(3,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1)) +-- !query analysis +Union false, false +:- Project [cast(CAST(20 AS DECIMAL(4,1))#x as decimal(9,6)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [(cast(10 as decimal(3,1)) / cast(10 as decimal(3,1))) AS (CAST(10 AS DECIMAL(3,1)) / CAST(10 AS DECIMAL(3,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1)) +-- !query analysis +Union false, false +:- Project [cast(CAST(20 AS DECIMAL(4,1))#x as decimal(6,2)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [(cast(10 as decimal(10,2)) % cast(3 as decimal(5,1))) AS (CAST(10 AS DECIMAL(10,2)) % CAST(3 AS DECIMAL(5,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1))) +-- !query analysis +Union false, false +:- Project [cast(CAST(20 AS DECIMAL(4,1))#x as decimal(6,2)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [pmod(cast(10 as decimal(10,2)), cast(3 as decimal(5,1))) AS pmod(CAST(10 AS DECIMAL(10,2)), CAST(3 AS DECIMAL(5,1)))#x] + +- OneRowRelation + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)) +-- !query analysis +Union false, false +:- Project [cast(CAST(20 AS DECIMAL(4,1))#x as decimal(21,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- Project [cast(20 as decimal(4,1)) AS CAST(20 AS DECIMAL(4,1))#x] +: +- OneRowRelation ++- Project [cast((CAST(10 AS DECIMAL(10,2)) div CAST(3 AS DECIMAL(5,1)))#xL as decimal(21,1)) AS (CAST(10 AS DECIMAL(10,2)) div CAST(3 AS DECIMAL(5,1)))#x] + +- Project [(cast(10 as decimal(10,2)) div cast(3 as decimal(5,1))) AS (CAST(10 AS DECIMAL(10,2)) div CAST(3 AS DECIMAL(5,1)))#xL] + +- OneRowRelation + + +-- !query +set spark.sql.decimalOperations.allowPrecisionLoss=false +-- !query analysis +SetCommand (spark.sql.decimalOperations.allowPrecisionLoss,Some(false)) + + +-- !query +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Repartition 1, false + +- Project [id#x, (a#x + b#x) AS (a + b)#x, (a#x - b#x) AS (a - b)#x, (a#x * b#x) AS (a * b)#x, (a#x / b#x) AS (a / b)#x] + +- SubqueryAlias spark_catalog.default.decimals_test + +- Relation spark_catalog.default.decimals_test[id#x,a#x,b#x] parquet + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query analysis +Sort [id#x ASC NULLS FIRST], true ++- Project [id#x, (a#x * cast(10 as decimal(2,0))) AS (a * 10)#x, (b#x / cast(10 as decimal(2,0))) AS (b / 10)#x] + +- SubqueryAlias spark_catalog.default.decimals_test + +- Relation spark_catalog.default.decimals_test[id#x,a#x,b#x] parquet + + +-- !query +select 10.3 * 3.0 +-- !query analysis +Project [(10.3 * 3.0) AS (10.3 * 3.0)#x] ++- OneRowRelation + + +-- !query +select 10.3000 * 3.0 +-- !query analysis +Project [(10.3000 * 3.0) AS (10.3000 * 3.0)#x] ++- OneRowRelation + + +-- !query +select 10.30000 * 30.0 +-- !query analysis +Project [(10.30000 * 30.0) AS (10.30000 * 30.0)#x] ++- OneRowRelation + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query analysis +Project [(10.300000000000000000 * 3.000000000000000000) AS (10.300000000000000000 * 3.000000000000000000)#x] ++- OneRowRelation + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query analysis +Project [(10.300000000000000000 * 3.0000000000000000000) AS (10.300000000000000000 * 3.0000000000000000000)#x] ++- OneRowRelation + + +-- !query +select 2.35E10 * 1.0 +-- !query analysis +Project [(2.35E10 * cast(1.0 as double)) AS (2.35E10 * 1.0)#x] ++- OneRowRelation + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query analysis +Project [((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000) AS ((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000)#x] ++- OneRowRelation + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query analysis +Project [((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000) AS ((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000)#x] ++- OneRowRelation + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query analysis +Project [(12345678901234567890.0 * 12345678901234567890.0) AS (12345678901234567890.0 * 12345678901234567890.0)#x] ++- OneRowRelation + + +-- !query +select 1e35BD / 0.1 +-- !query analysis +Project [(100000000000000000000000000000000000 / 0.1) AS (100000000000000000000000000000000000 / 0.1)#x] ++- OneRowRelation + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query analysis +Project [(1234567890123456789000000000000 * 12345678901234567890000000) AS (1234567890123456789000000000000 * 12345678901234567890000000)#x] ++- OneRowRelation + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query analysis +Project [(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345) AS (12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345)#x] ++- OneRowRelation + + -- !query select 123456789123456789.1234567890 * 1.123456789123456789 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers-disabled.sql.out rename to sql/core/src/test/resources/sql-tests/analyzer-results/nonansi/double-quoted-identifiers.sql.out diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out index 93456003254b8..dfba3688f0b7d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/union.sql.out @@ -40,7 +40,7 @@ Project [c1#x, c2#x] -- !query SELECT * -FROM (SELECT * FROM t1 +FROM (SELECT * FROM t1 where c1 = 1 UNION ALL SELECT * FROM t2 UNION ALL @@ -52,11 +52,12 @@ Project [c1#x, c2#xL] :- Union false, false : :- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : : +- Project [c1#x, c2#x] - : : +- SubqueryAlias t1 - : : +- View (`t1`, [c1#x, c2#x]) - : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as string) AS c2#x] - : : +- SubqueryAlias tbl - : : +- LocalRelation [c1#x, c2#x] + : : +- Filter (c1#x = 1) + : : +- SubqueryAlias t1 + : : +- View (`t1`, [c1#x, c2#x]) + : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as string) AS c2#x] + : : +- SubqueryAlias tbl + : : +- LocalRelation [c1#x, c2#x] : +- Project [cast(c1#x as decimal(11,1)) AS c1#x, cast(c2#x as bigint) AS c2#xL] : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 diff --git a/sql/core/src/test/resources/sql-tests/inputs/conditional-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/conditional-functions.sql new file mode 100644 index 0000000000000..c7a4b055f024e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/conditional-functions.sql @@ -0,0 +1,38 @@ +-- Tests for conditional functions + +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2); + +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t; +SELECT nanvl(c2, 1/0) FROM conditional_t; +SELECT nanvl(1-0, 1/0) FROM conditional_t; + +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t; +SELECT if(1 == 1, 1, 1/0); +SELECT if(1 != 1, 1/0, 1); + +SELECT coalesce(c2, 1/0) from conditional_t; +SELECT coalesce(1, 1/0); +SELECT coalesce(null, 1, 1/0); + +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t; +SELECT case when 1 < 2 then 1 else 1/0 end; +SELECT case when 1 > 2 then 1/0 else 1 end; + +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null); + +SELECT nullifzero('abc'); + +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)); + +SELECT zeroifnull('abc'); + +DROP TABLE conditional_t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-enabled.sql b/sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers-enabled.sql similarity index 100% rename from sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-enabled.sql rename to sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers-enabled.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql index 4dc6d3d0189cc..451f745a97ee6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql @@ -12,7 +12,7 @@ SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROU SELECT i AS i, COUNT(i) ci, f AS f, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, i, 2, ci, 3, f, 4, sif; -SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff +SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), octet_length(d), size(a) FROM stuff GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2; -- unlike Mosha, I'm failing this case because IMO it is too implicit to automatically group by i. diff --git a/sql/core/src/test/resources/sql-tests/inputs/keywords-enforced.sql b/sql/core/src/test/resources/sql-tests/inputs/keywords-enforced.sql new file mode 100644 index 0000000000000..b0c6e5929b18d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/keywords-enforced.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enforceReservedKeywords = true +--IMPORT keywords.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql index c7a4b055f024e..c4b3418cc6d89 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/nonansi/conditional-functions.sql @@ -1,38 +1 @@ --- Tests for conditional functions - -CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2); - -SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t; -SELECT nanvl(c2, 1/0) FROM conditional_t; -SELECT nanvl(1-0, 1/0) FROM conditional_t; - -SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t; -SELECT if(1 == 1, 1, 1/0); -SELECT if(1 != 1, 1/0, 1); - -SELECT coalesce(c2, 1/0) from conditional_t; -SELECT coalesce(1, 1/0); -SELECT coalesce(null, 1, 1/0); - -SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t; -SELECT case when 1 < 2 then 1 else 1/0 end; -SELECT case when 1 > 2 then 1/0 else 1 end; - -SELECT nullifzero(0), - nullifzero(cast(0 as tinyint)), - nullifzero(cast(0 as bigint)), - nullifzero('0'), - nullifzero(0.0), - nullifzero(1), - nullifzero(null); - -SELECT nullifzero('abc'); - -SELECT zeroifnull(null), - zeroifnull(1), - zeroifnull(cast(1 as tinyint)), - zeroifnull(cast(1 as bigint)); - -SELECT zeroifnull('abc'); - -DROP TABLE conditional_t; +--IMPORT conditional-functions.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql index c447511ba6055..7d6c336df4528 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql @@ -1,41 +1 @@ --- SPARK-23179: SQL ANSI 2011 states that in case of overflow during arithmetic operations, --- an exception should be thrown instead of returning NULL. --- This is what most of the SQL DBs do (e.g. SQLServer, DB2). - --- tests for decimals handling in operations -create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet; - -insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), - (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789); - --- test operations between decimals and constants -select id, a*10, b/10 from decimals_test order by id; - --- test operations on constants -select 10.3 * 3.0; -select 10.3000 * 3.0; -select 10.30000 * 30.0; -select 10.300000000000000000 * 3.000000000000000000; -select 10.300000000000000000 * 3.0000000000000000000; - --- arithmetic operations causing an overflow throw exception -select (5e36BD + 0.1) + 5e36BD; -select (-4e36BD - 0.1) - 7e36BD; -select 12345678901234567890.0 * 12345678901234567890.0; -select 1e35BD / 0.1; - --- arithmetic operations causing a precision loss throw exception -select 123456789123456789.1234567890 * 1.123456789123456789; -select 123456789123456789.1234567890 * 1.123456789123456789; -select 12345678912345.123456789123 / 0.000000012345678; - -select 1.0123456789012345678901234567890123456e36BD / 0.1; -select 1.0123456789012345678901234567890123456e35BD / 1.0; -select 1.0123456789012345678901234567890123456e34BD / 1.0; -select 1.0123456789012345678901234567890123456e33BD / 1.0; -select 1.0123456789012345678901234567890123456e32BD / 1.0; -select 1.0123456789012345678901234567890123456e31BD / 1.0; -select 1.0123456789012345678901234567890123456e31BD / 0.1; -select 1.0123456789012345678901234567890123456e31BD / 10.0; - -drop table decimals_test; +--IMPORT decimalArithmeticOperations.sql \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql deleted file mode 100644 index b8ff8cdb81376..0000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers-disabled.sql +++ /dev/null @@ -1,2 +0,0 @@ ---SET spark.sql.ansi.doubleQuotedIdentifiers=false ---IMPORT double-quoted-identifiers.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers.sql new file mode 100644 index 0000000000000..7ccd294e709b6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/nonansi/double-quoted-identifiers.sql @@ -0,0 +1 @@ +--IMPORT double-quoted-identifiers.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql b/sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql index b0c6e5929b18d..ebc9345c3a738 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/nonansi/keywords.sql @@ -1,2 +1 @@ ---SET spark.sql.ansi.enforceReservedKeywords = true --IMPORT keywords.sql diff --git a/sql/core/src/test/resources/sql-tests/inputs/union.sql b/sql/core/src/test/resources/sql-tests/inputs/union.sql index 8a5b6c50fc1e3..ab81cc7bbabb6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/union.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/union.sql @@ -9,7 +9,7 @@ FROM (SELECT * FROM t1 -- Type Coerced Union SELECT * -FROM (SELECT * FROM t1 +FROM (SELECT * FROM t1 where c1 = 1 UNION ALL SELECT * FROM t2 UNION ALL diff --git a/sql/core/src/test/resources/sql-tests/results/conditional-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/conditional-functions.sql.out new file mode 100644 index 0000000000000..aa8a600f87560 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/conditional-functions.sql.out @@ -0,0 +1,202 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(c2, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(1-0, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t +-- !query schema +struct<(IF((c2 >= 0), (1 - 0), (1 / 0))):double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(1 == 1, 1, 1/0) +-- !query schema +struct<(IF((1 = 1), 1, (1 / 0))):double> +-- !query output +1.0 + + +-- !query +SELECT if(1 != 1, 1/0, 1) +-- !query schema +struct<(IF((NOT (1 = 1)), (1 / 0), 1)):double> +-- !query output +1.0 + + +-- !query +SELECT coalesce(c2, 1/0) from conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT coalesce(1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT coalesce(null, 1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t +-- !query schema +struct= 0) THEN 1 ELSE (1 / 0) END:double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT case when 1 < 2 then 1 else 1/0 end +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when 1 > 2 then 1/0 else 1 end +-- !query schema +struct 2) THEN (1 / 0) ELSE 1 END:double> +-- !query output +1.0 + + +-- !query +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null) +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL 1 NULL + + +-- !query +SELECT nullifzero('abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "nullifzero('abc')" + } ] +} + + +-- !query +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)) +-- !query schema +struct +-- !query output +0 1 1 1 + + +-- !query +SELECT zeroifnull('abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "zeroifnull('abc')" + } ] +} + + +-- !query +DROP TABLE conditional_t +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers-enabled.sql.out similarity index 66% rename from sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out rename to sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers-enabled.sql.out index 81a98a60590f0..2444c399a87ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers-enabled.sql.out @@ -4,14 +4,20 @@ SELECT 1 FROM "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "\"not_exist\"" + } ] } @@ -20,13 +26,12 @@ USE SCHEMA "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" + "schemaName" : "`spark_catalog`.`not_exist`" } } @@ -36,14 +41,20 @@ ALTER TABLE "not_exist" ADD COLUMN not_exist int -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "\"not_exist\"" + } ] } @@ -52,14 +63,20 @@ ALTER TABLE not_exist ADD COLUMN "not_exist" int -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] } @@ -68,14 +85,20 @@ SELECT 1 AS "not_exist" FROM not_exist -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 38, + "fragment" : "not_exist" + } ] } @@ -84,14 +107,20 @@ SELECT 1 FROM not_exist AS X("hello") -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", "messageParameters" : { - "error" : "'\"hello\"'", - "hint" : "" - } + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] } @@ -100,14 +129,21 @@ SELECT "not_exist"() -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "\"not_exist\"()" + } ] } @@ -116,14 +152,21 @@ SELECT "not_exist".not_exist() -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException +org.apache.spark.sql.AnalysisException { - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", "messageParameters" : { - "error" : "'\"not_exist\"'", - "hint" : "" - } + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "\"not_exist\".not_exist()" + } ] } @@ -301,9 +344,23 @@ org.apache.spark.sql.AnalysisException -- !query SELECT "hello" -- !query schema -struct +struct<> -- !query output -hello +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "\"hello\"" + } ] +} -- !query @@ -311,7 +368,15 @@ CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} -- !query @@ -319,15 +384,30 @@ DROP VIEW v -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`v`" + } +} -- !query SELECT INTERVAL "1" YEAR -- !query schema -struct +struct<> -- !query output -1-0 +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"1\"'", + "hint" : "" + } +} -- !query @@ -367,15 +447,7 @@ CREATE SCHEMA "myschema" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myschema\"'", - "hint" : "" - } -} + -- !query @@ -384,31 +456,15 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myview\"'", - "hint" : "" - } -} + -- !query SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"a2\"'", - "hint" : "" - } -} +1 -- !query @@ -416,15 +472,7 @@ DROP TABLE "myview" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myview\"'", - "hint" : "" - } -} + -- !query @@ -432,12 +480,4 @@ DROP SCHEMA "myschema" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.parser.ParseException -{ - "errorClass" : "PARSE_SYNTAX_ERROR", - "sqlState" : "42601", - "messageParameters" : { - "error" : "'\"myschema\"'", - "hint" : "" - } -} + diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out index d6735f620a638..f0708c56224fe 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out @@ -41,28 +41,14 @@ struct -- !query -SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff +SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), octet_length(d), size(a) FROM stuff GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2 -- !query schema -struct<> +struct<(i + 1):int,(f / i):decimal(17,15),substring(s, 2, 3):string,extract(year FROM t):int,octet_length(d):int,size(a):int> -- !query output -org.apache.spark.SparkNumberFormatException -{ - "errorClass" : "CAST_INVALID_INPUT", - "sqlState" : "22018", - "messageParameters" : { - "expression" : "'42.0'", - "sourceType" : "\"STRING\"", - "targetType" : "\"BIGINT\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 64, - "stopIndex" : 68, - "fragment" : "d / 2" - } ] -} +43 0.232142857142857 ell 1970 5 3 +43 0.318333333333333 est 1970 10 3 +1338 0.000923335826477 h n 2000 4 3 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out new file mode 100644 index 0000000000000..b2331ec4ab804 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out @@ -0,0 +1,453 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * from SQL_KEYWORDS() +-- !query schema +struct +-- !query output +ADD false +AFTER false +AGGREGATE false +ALL true +ALTER false +ALWAYS false +ANALYZE false +AND true +ANTI false +ANY true +ANY_VALUE false +ARCHIVE false +ARRAY false +AS true +ASC false +AT false +AUTHORIZATION true +BEGIN false +BETWEEN false +BIGINT false +BINARY false +BINDING false +BOOLEAN false +BOTH true +BUCKET false +BUCKETS false +BY false +BYTE false +CACHE false +CALL true +CALLED false +CASCADE false +CASE true +CAST true +CATALOG false +CATALOGS false +CHANGE false +CHAR false +CHARACTER false +CHECK true +CLEAR false +CLUSTER false +CLUSTERED false +CODEGEN false +COLLATE true +COLLATION true +COLLECTION false +COLUMN true +COLUMNS false +COMMENT false +COMMIT false +COMPACT false +COMPACTIONS false +COMPENSATION false +COMPUTE false +CONCATENATE false +CONSTRAINT true +CONTAINS false +COST false +CREATE true +CROSS true +CUBE false +CURRENT false +CURRENT_DATE true +CURRENT_TIME true +CURRENT_TIMESTAMP true +CURRENT_USER true +DATA false +DATABASE false +DATABASES false +DATE false +DATEADD false +DATEDIFF false +DATE_ADD false +DATE_DIFF false +DAY false +DAYOFYEAR false +DAYS false +DBPROPERTIES false +DEC false +DECIMAL false +DECLARE false +DEFAULT false +DEFINED false +DEFINER false +DELETE false +DELIMITED false +DESC false +DESCRIBE false +DETERMINISTIC false +DFS false +DIRECTORIES false +DIRECTORY false +DISTINCT true +DISTRIBUTE false +DIV false +DO false +DOUBLE false +DROP false +ELSE true +END true +ESCAPE true +ESCAPED false +EVOLUTION false +EXCEPT true +EXCHANGE false +EXCLUDE false +EXECUTE true +EXISTS false +EXPLAIN false +EXPORT false +EXTENDED false +EXTERNAL false +EXTRACT false +FALSE true +FETCH true +FIELDS false +FILEFORMAT false +FILTER true +FIRST false +FLOAT false +FOLLOWING false +FOR true +FOREIGN true +FORMAT false +FORMATTED false +FROM true +FULL true +FUNCTION false +FUNCTIONS false +GENERATED false +GLOBAL false +GRANT true +GROUP true +GROUPING false +HAVING true +HOUR false +HOURS false +IDENTIFIER false +IDENTITY false +IF false +IGNORE false +ILIKE false +IMMEDIATE false +IMPORT false +IN true +INCLUDE false +INCREMENT false +INDEX false +INDEXES false +INNER true +INPATH false +INPUT false +INPUTFORMAT false +INSERT false +INT false +INTEGER false +INTERSECT true +INTERVAL false +INTO true +INVOKER false +IS true +ITEMS false +ITERATE false +JOIN true +KEYS false +LANGUAGE false +LAST false +LATERAL true +LAZY false +LEADING true +LEAVE false +LEFT true +LIKE false +LIMIT false +LINES false +LIST false +LOAD false +LOCAL false +LOCATION false +LOCK false +LOCKS false +LOGICAL false +LONG false +LOOP false +MACRO false +MAP false +MATCHED false +MERGE false +MICROSECOND false +MICROSECONDS false +MILLISECOND false +MILLISECONDS false +MINUS false +MINUTE false +MINUTES false +MODIFIES false +MONTH false +MONTHS false +MSCK false +NAME false +NAMESPACE false +NAMESPACES false +NANOSECOND false +NANOSECONDS false +NATURAL true +NO false +NONE false +NOT true +NULL true +NULLS false +NUMERIC false +OF false +OFFSET true +ON true +ONLY true +OPTION false +OPTIONS false +OR true +ORDER true +OUT false +OUTER true +OUTPUTFORMAT false +OVER false +OVERLAPS true +OVERLAY false +OVERWRITE false +PARTITION false +PARTITIONED false +PARTITIONS false +PERCENT false +PIVOT false +PLACING false +POSITION false +PRECEDING false +PRIMARY true +PRINCIPALS false +PROPERTIES false +PURGE false +QUARTER false +QUERY false +RANGE false +READS false +REAL false +RECORDREADER false +RECORDWRITER false +RECOVER false +REDUCE false +REFERENCES true +REFRESH false +RENAME false +REPAIR false +REPEAT false +REPEATABLE false +REPLACE false +RESET false +RESPECT false +RESTRICT false +RETURN false +RETURNS false +REVOKE false +RIGHT true +ROLE false +ROLES false +ROLLBACK false +ROLLUP false +ROW false +ROWS false +SCHEMA false +SCHEMAS false +SECOND false +SECONDS false +SECURITY false +SELECT true +SEMI false +SEPARATED false +SERDE false +SERDEPROPERTIES false +SESSION_USER true +SET false +SETS false +SHORT false +SHOW false +SINGLE false +SKEWED false +SMALLINT false +SOME true +SORT false +SORTED false +SOURCE false +SPECIFIC false +SQL true +START false +STATISTICS false +STORED false +STRATIFY false +STRING false +STRUCT false +SUBSTR false +SUBSTRING false +SYNC false +SYSTEM_TIME false +SYSTEM_VERSION false +TABLE true +TABLES false +TABLESAMPLE false +TARGET false +TBLPROPERTIES false +TERMINATED false +THEN true +TIME true +TIMEDIFF false +TIMESTAMP false +TIMESTAMPADD false +TIMESTAMPDIFF false +TIMESTAMP_LTZ false +TIMESTAMP_NTZ false +TINYINT false +TO true +TOUCH false +TRAILING true +TRANSACTION false +TRANSACTIONS false +TRANSFORM false +TRIM false +TRUE false +TRUNCATE false +TRY_CAST false +TYPE false +UNARCHIVE false +UNBOUNDED false +UNCACHE false +UNION true +UNIQUE true +UNKNOWN true +UNLOCK false +UNPIVOT false +UNSET false +UNTIL false +UPDATE false +USE false +USER true +USING true +VALUES false +VAR false +VARCHAR false +VARIABLE false +VARIANT false +VERSION false +VIEW false +VIEWS false +VOID false +WEEK false +WEEKS false +WHEN true +WHERE true +WHILE false +WINDOW false +WITH true +WITHIN true +X false +YEAR false +YEARS false +ZONE false + + +-- !query +SELECT keyword from SQL_KEYWORDS() WHERE reserved +-- !query schema +struct +-- !query output +ALL +AND +ANY +AS +AUTHORIZATION +BOTH +CALL +CASE +CAST +CHECK +COLLATE +COLLATION +COLUMN +CONSTRAINT +CREATE +CROSS +CURRENT_DATE +CURRENT_TIME +CURRENT_TIMESTAMP +CURRENT_USER +DISTINCT +ELSE +END +ESCAPE +EXCEPT +EXECUTE +FALSE +FETCH +FILTER +FOR +FOREIGN +FROM +FULL +GRANT +GROUP +HAVING +IN +INNER +INTERSECT +INTO +IS +JOIN +LATERAL +LEADING +LEFT +NATURAL +NOT +NULL +OFFSET +ON +ONLY +OR +ORDER +OUTER +OVERLAPS +PRIMARY +REFERENCES +RIGHT +SELECT +SESSION_USER +SOME +SQL +TABLE +THEN +TIME +TO +TRAILING +UNION +UNIQUE +UNKNOWN +USER +USING +WHEN +WHERE +WITH +WITHIN diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out index 8074a7bf2ac6d..8276168d8bb87 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out @@ -1,4 +1,36 @@ -- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b +-- !query schema +struct<> +-- !query output + + + +-- !query +select a / b from t +-- !query schema +struct<(a / b):decimal(8,6)> +-- !query output +NULL + + +-- !query +select a % b from t +-- !query schema +struct<(a % b):decimal(1,1)> +-- !query output +NULL + + +-- !query +select pmod(a, b) from t +-- !query schema +struct +-- !query output +NULL + + -- !query create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet -- !query schema @@ -16,6 +48,17 @@ struct<> +-- !query +select id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct +-- !query output +1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 +2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 +3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 +4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 + + -- !query select id, a*10, b/10 from decimals_test order by id -- !query schema @@ -67,6 +110,14 @@ struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,34)> 30.9000000000000000000000000000000000 +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + -- !query select (5e36BD + 0.1) + 5e36BD -- !query schema @@ -100,11 +151,19 @@ NULL -- !query -select 123456789123456789.1234567890 * 1.123456789123456789 +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD -- !query schema -struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> +struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> -- !query output -138698367904130467.654320988515622621 +NULL + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,6)> +-- !query output +10012345678912345678912345678911.246907 -- !query @@ -123,10 +182,229 @@ struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)> 1000000073899961059796.725866332 +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +1.000000 +20.000000 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1))) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +20.0 +3.0 + + +-- !query +set spark.sql.decimalOperations.allowPrecisionLoss=false +-- !query schema +struct +-- !query output +spark.sql.decimalOperations.allowPrecisionLoss false + + +-- !query +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct +-- !query output +1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 +2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 +3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 +4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000000 99.9000000000000000000 +2 123451.230000000000000000 1234.5123000000000000000 +3 1.234567891011000000 123.4100000000000000000 +4 1234567891234567890.000000000000000000 0.1123456789123456789 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,36)> +-- !query output +30.900000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,37)> +-- !query output +NULL + + +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +-- !query output +NULL + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<(100000000000000000000000000000000000 / 0.1):decimal(38,3)> +-- !query output +NULL + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query schema +struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> +-- !query output +NULL + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,7)> +-- !query output +NULL + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,28)> +-- !query output +NULL + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,18)> +-- !query output +NULL + + -- !query select 1.0123456789012345678901234567890123456e36BD / 0.1 -- !query schema -struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,6)> +struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,2)> -- !query output NULL @@ -134,7 +412,7 @@ NULL -- !query select 1.0123456789012345678901234567890123456e35BD / 1.0 -- !query schema -struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,6)> +struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,3)> -- !query output NULL @@ -142,25 +420,25 @@ NULL -- !query select 1.0123456789012345678901234567890123456e34BD / 1.0 -- !query schema -struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,6)> +struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,3)> -- !query output -NULL +10123456789012345678901234567890123.456 -- !query select 1.0123456789012345678901234567890123456e33BD / 1.0 -- !query schema -struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,6)> +struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,4)> -- !query output -NULL +1012345678901234567890123456789012.3456 -- !query select 1.0123456789012345678901234567890123456e32BD / 1.0 -- !query schema -struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,6)> +struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,5)> -- !query output -NULL +101234567890123456789012345678901.23456 -- !query @@ -182,9 +460,9 @@ NULL -- !query select 1.0123456789012345678901234567890123456e31BD / 10.0 -- !query schema -struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,6)> +struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,7)> -- !query output -1012345678901234567890123456789.012346 +1012345678901234567890123456789.0123456 -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers.sql.out similarity index 100% rename from sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers-disabled.sql.out rename to sql/core/src/test/resources/sql-tests/results/nonansi/double-quoted-identifiers.sql.out diff --git a/sql/core/src/test/resources/sql-tests/results/union.sql.out b/sql/core/src/test/resources/sql-tests/results/union.sql.out index 3825470777bdb..d7db2163f8b87 100644 --- a/sql/core/src/test/resources/sql-tests/results/union.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/union.sql.out @@ -31,7 +31,7 @@ struct -- !query SELECT * -FROM (SELECT * FROM t1 +FROM (SELECT * FROM t1 where c1 = 1 UNION ALL SELECT * FROM t2 UNION ALL @@ -52,8 +52,8 @@ org.apache.spark.SparkNumberFormatException "objectType" : "", "objectName" : "", "startIndex" : 1, - "stopIndex" : 120, - "fragment" : "SELECT *\nFROM (SELECT * FROM t1\n UNION ALL\n SELECT * FROM t2\n UNION ALL\n SELECT * FROM t2)" + "stopIndex" : 133, + "fragment" : "SELECT *\nFROM (SELECT * FROM t1 where c1 = 1\n UNION ALL\n SELECT * FROM t2\n UNION ALL\n SELECT * FROM t2)" } ] } From b74aa8c2d6e7c35fda3acb0482943e1604b717e1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2024 15:01:26 +0800 Subject: [PATCH 11/58] [SPARK-50340][SQL] Unwrap UDT in INSERT input query ### What changes were proposed in this pull request? This is a new approach for https://github.com/apache/spark/pull/48251 . We should unwrap UDT in the INSERT input query, so that we can treat them the same as the underlying SQL type, and allow writing them to the target table if the SQL type is compatible with table schema. ### Why are the changes needed? allow writing UDT to the table that has the same SQL type. ### Does this PR introduce _any_ user-facing change? Yes, writing UDT to table with same SQL type will be allowed. ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48881 from cloud-fan/cast. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../analysis/TableOutputResolver.scala | 38 +++++++++++++- .../apache/spark/sql/SQLInsertTestSuite.scala | 51 +++++++++++++++++++ 2 files changed, 87 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala index e2e9bde856781..4a6504666d41f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy -import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, IntegralType, MapType, StructType} +import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, IntegralType, MapType, StructType, UserDefinedType} object TableOutputResolver extends SQLConfHelper with Logging { @@ -539,7 +539,8 @@ object TableOutputResolver extends SQLConfHelper with Logging { } } else { val nullCheckedQueryExpr = checkNullability(queryExpr, tableAttr, conf, colPath) - val casted = cast(nullCheckedQueryExpr, attrTypeWithoutCharVarchar, conf, colPath.quoted) + val udtUnwrapped = unwrapUDT(nullCheckedQueryExpr) + val casted = cast(udtUnwrapped, attrTypeWithoutCharVarchar, conf, colPath.quoted) val exprWithStrLenCheck = if (conf.charVarcharAsString || !attrTypeHasCharVarchar) { casted } else { @@ -558,6 +559,39 @@ object TableOutputResolver extends SQLConfHelper with Logging { if (canWriteExpr) outputField else None } + private def unwrapUDT(expr: Expression): Expression = expr.dataType match { + case ArrayType(et, containsNull) => + val param = NamedLambdaVariable("element", et, containsNull) + val func = LambdaFunction(unwrapUDT(param), Seq(param)) + ArrayTransform(expr, func) + + case MapType(kt, vt, valueContainsNull) => + val keyParam = NamedLambdaVariable("key", kt, nullable = false) + val valueParam = NamedLambdaVariable("value", vt, valueContainsNull) + val keyFunc = LambdaFunction(unwrapUDT(keyParam), Seq(keyParam)) + val valueFunc = LambdaFunction(unwrapUDT(valueParam), Seq(valueParam)) + val newKeys = ArrayTransform(MapKeys(expr), keyFunc) + val newValues = ArrayTransform(MapValues(expr), valueFunc) + MapFromArrays(newKeys, newValues) + + case st: StructType => + val newFieldExprs = st.indices.map { i => + unwrapUDT(GetStructField(expr, i)) + } + val struct = CreateNamedStruct(st.zip(newFieldExprs).flatMap { + case (field, newExpr) => Seq(Literal(field.name), newExpr) + }) + if (expr.nullable) { + If(IsNull(expr), Literal(null, struct.dataType), struct) + } else { + struct + } + + case _: UserDefinedType[_] => UnwrapUDT(expr) + + case _ => expr + } + private def cast( expr: Expression, expectedType: DataType, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 170105200f1d1..a4e39df6c632d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} +import org.apache.spark.sql.types.{DataType, IntegerType, SQLUserDefinedType, UserDefinedType} import org.apache.spark.unsafe.types.UTF8String /** @@ -549,6 +550,56 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP assert(reusedExchanges.size == 1) } } + + test("SPARK-50340: unwrap UDT before insertion") { + withTempView("v") { + Seq((1, MyInt(2))).toDF("c1", "c2").createTempView("v") + + withTable("t") { + createTable("t", Seq("c1", "c2"), Seq("int", "int")) + sql("INSERT INTO t SELECT * FROM v") + checkAnswer(spark.table("t"), Row(1, 2)) + } + + // can upcast UDT input + withTable("t") { + createTable("t", Seq("c1", "c2"), Seq("int", "long")) + sql("INSERT INTO t SELECT * FROM v") + checkAnswer(spark.table("t"), Row(1, 2L)) + } + + // Array of UDT + withTable("t") { + createTable("t", Seq("c1", "c2"), Seq("int", "array")) + sql("INSERT INTO t SELECT c1, array(c2) FROM v") + checkAnswer(spark.table("t"), Row(1, Seq(2))) + } + + // Map of UDT + withTable("t") { + createTable("t", Seq("c1", "c2"), Seq("int", "map")) + sql("INSERT INTO t SELECT c1, map(c2, c2) FROM v") + checkAnswer(spark.table("t"), Row(1, Map(2 -> 2))) + } + + // Struct of UDT + withTable("t") { + createTable("t", Seq("c1", "c2"), Seq("int", "struct")) + sql("INSERT INTO t SELECT c1, struct(c2 as f1, c2 as f2) FROM v") + checkAnswer(spark.table("t"), Row(1, Row(2, 2))) + } + } + } +} + +@SQLUserDefinedType(udt = classOf[MyIntUDT]) +private case class MyInt(i: Int) + +private class MyIntUDT extends UserDefinedType[MyInt] { + override def sqlType: DataType = IntegerType + override def serialize(obj: MyInt): Any = obj.i + override def deserialize(datum: Any): MyInt = MyInt(datum.asInstanceOf[Int]) + override def userClass: Class[MyInt] = classOf[MyInt] } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { From 87a5b37ec3c4b383a5938144612c07187d597ff8 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 19 Nov 2024 16:54:29 +0800 Subject: [PATCH 12/58] [SPARK-50313][SQL][TESTS][FOLLOWUP] Regenerate golden files for Java 21 ### What changes were proposed in this pull request? This pr uses the command ``` java -version openjdk version "21.0.5" 2024-10-15 LTS OpenJDK Runtime Environment Zulu21.38+21-CA (build 21.0.5+11-LTS) OpenJDK 64-Bit Server VM Zulu21.38+21-CA (build 21.0.5+11-LTS, mixed mode, sharing) SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite" ``` to regenerate golden files for Java 21. ### Why are the changes needed? Regenerate golden files for Java 21. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #48886 from LuciferYang/SPARK-50313-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: yangjie01 --- .../nonansi/try_aggregates.sql.out.java21 | 299 +++++++++ .../nonansi/try_arithmetic.sql.out.java21 | 569 ++++++++++++++++++ .../results/try_aggregates.sql.out.java21 | 148 ++++- .../results/try_arithmetic.sql.out.java21 | 244 +++++++- 4 files changed, 1214 insertions(+), 46 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out.java21 create mode 100644 sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out.java21 new file mode 100644 index 0000000000000..7affe568234f9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out.java21 @@ -0,0 +1,299 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +30.0 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-2 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:02.000000000 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +10.0 + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +10.00000 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +12.5 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +4.611686018427388E18 + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-4.611686018427388E18 + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 new file mode 100644 index 0000000000000..002a0dfcf37ef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 @@ -0,0 +1,569 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_add(2147483647, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(2147483647, decimal(1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(2147483647, "1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483647 + + +-- !query +SELECT try_add(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775807 + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_add(interval 2 year, interval 2 second)" + } ] +} + + +-- !query +SELECT try_add(interval 2147483647 month, interval 2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, 0.5) +-- !query schema +struct +-- !query output +2.000000 + + +-- !query +SELECT try_divide(1, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-4.656612873077393E-10 + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-1.0842021724855044E-19 + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, decimal(0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, "0") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(2147483647, decimal(-1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(2147483647, "-1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2, 3) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2147483647, decimal(-2)) +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(2147483647, "-2") +-- !query schema +struct +-- !query output +-4.294967294E9 + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query schema +struct +-- !query output +0-0 + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out.java21 index 7affe568234f9..9d3c97baecabd 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out.java21 +++ b/sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out.java21 @@ -82,33 +82,91 @@ NULL -- !query SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct +struct<> -- !query output --9223372036854775806 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} -- !query @@ -232,33 +290,91 @@ NULL -- !query SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} -- !query SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) -- !query schema -struct +struct<> -- !query output --4.611686018427388E18 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 index 002a0dfcf37ef..acf6e70a50dea 100644 --- a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 +++ b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 @@ -26,9 +26,9 @@ struct -- !query SELECT try_add(2147483647, "1") -- !query schema -struct +struct -- !query output -2.147483648E9 +2147483648 -- !query @@ -58,25 +58,71 @@ NULL -- !query SELECT try_add(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --2147483647 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 33, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_add(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --9223372036854775807 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 45, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_add(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -244,25 +290,71 @@ NULL -- !query SELECT try_divide(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --4.656612873077393E-10 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 36, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_divide(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --1.0842021724855044E-19 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 48, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_divide(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 30, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -356,9 +448,9 @@ struct -- !query SELECT try_subtract(2147483647, "-1") -- !query schema -struct +struct -- !query output -2.147483648E9 +2147483648 -- !query @@ -388,25 +480,71 @@ NULL -- !query SELECT try_subtract(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 38, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_subtract(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_subtract(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} -- !query @@ -468,9 +606,9 @@ struct -- !query SELECT try_multiply(2147483647, "-2") -- !query schema -struct +struct -- !query output --4.294967294E9 +-4294967294 -- !query @@ -500,25 +638,71 @@ NULL -- !query SELECT try_multiply(1, (2147483647 + 1)) -- !query schema -struct +struct<> -- !query output --2147483648 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 38, + "fragment" : "2147483647 + 1" + } ] +} -- !query SELECT try_multiply(1L, (9223372036854775807L + 1L)) -- !query schema -struct +struct<> -- !query output --9223372036854775808 +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} -- !query SELECT try_multiply(1, 1.0 / 0.0) -- !query schema -struct +struct<> -- !query output -NULL +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} -- !query From f1b68d897e49e77308fb75bb60d054db10f6a90c Mon Sep 17 00:00:00 2001 From: Ole Sasse Date: Tue, 19 Nov 2024 20:25:52 +0800 Subject: [PATCH 13/58] [SPARK-50315][SQL] Support custom metrics for V1Fallback writes ### What changes were proposed in this pull request? Support for custom metrics for V1Fallback writers (AppendDataExecV1, OverwriteByExpressionExecV1) ### Why are the changes needed? * Add the custom metrics of the V1Write as metrics to the V1FallbackWriters implementations * Publish the metrics from reportDriverMetrics ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added a new test on top of mocked implementations ### Was this patch authored or co-authored using generative AI tooling? No Closes #48867 from olaky/sc-50315-metrics-for-v1-fallback-writers. Authored-by: Ole Sasse Signed-off-by: Wenchen Fan --- .../datasources/v2/V1FallbackWriters.scala | 23 ++++++-- .../org/apache/spark/sql/QueryTest.scala | 25 ++++++++- .../sql/connector/V1WriteFallbackSuite.scala | 54 +++++++++++++++++++ 3 files changed, 96 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala index 6f83b82785955..801151c51206d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V1FallbackWriters.scala @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.SupportsWrite import org.apache.spark.sql.connector.write.V1Write -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.sources.InsertableRelation /** @@ -58,14 +59,27 @@ case class OverwriteByExpressionExecV1( sealed trait V1FallbackWriters extends LeafV2CommandExec with SupportsV1Write { override def output: Seq[Attribute] = Nil + override val metrics: Map[String, SQLMetric] = + write.supportedCustomMetrics().map { customMetric => + customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric) + }.toMap + def table: SupportsWrite def refreshCache: () => Unit def write: V1Write override def run(): Seq[InternalRow] = { - val writtenRows = writeWithV1(write.toInsertableRelation) + writeWithV1(write.toInsertableRelation) refreshCache() - writtenRows + + write.reportDriverMetrics().foreach { customTaskMetric => + metrics.get(customTaskMetric.name()).foreach(_.set(customTaskMetric.value())) + } + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + + Nil } } @@ -75,8 +89,7 @@ sealed trait V1FallbackWriters extends LeafV2CommandExec with SupportsV1Write { trait SupportsV1Write extends SparkPlan { def plan: LogicalPlan - protected def writeWithV1(relation: InsertableRelation): Seq[InternalRow] = { + protected def writeWithV1(relation: InsertableRelation): Unit = { relation.insert(Dataset.ofRows(session, plan), overwrite = false) - Nil } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index f5ba655e3e85f..30180d48da71a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -27,8 +27,9 @@ import org.scalatest.Assertions import org.apache.spark.sql.catalyst.ExtendedAnalysisException import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.storage.StorageLevel import org.apache.spark.util.ArrayImplicits._ @@ -447,6 +448,28 @@ object QueryTest extends Assertions { case None => } } + + def withPhysicalPlansCaptured(spark: SparkSession, thunk: => Unit): Seq[SparkPlan] = { + var capturedPlans = Seq.empty[SparkPlan] + + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + capturedPlans = capturedPlans :+ qe.executedPlan + } + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} + } + + spark.sparkContext.listenerBus.waitUntilEmpty(15000) + spark.listenerManager.register(listener) + try { + thunk + spark.sparkContext.listenerBus.waitUntilEmpty(15000) + } finally { + spark.listenerManager.unregister(listener) + } + + capturedPlans + } } class QueryTestSuite extends QueryTest with test.SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala index ad31cf84eeb3f..04fc7e23ebb24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, SaveMode, SparkSession, SQLContext} +import org.apache.spark.sql.QueryTest.withPhysicalPlansCaptured import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule @@ -31,9 +32,12 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable, SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.expressions.{FieldReference, IdentityTransform, Transform} +import org.apache.spark.sql.connector.metric.{CustomMetric, CustomSumMetric, CustomTaskMetric} import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, V1Scan} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl, SupportsOverwrite, SupportsTruncate, V1Write, WriteBuilder} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.execution.datasources.v2.{AppendDataExecV1, OverwriteByExpressionExecV1} import org.apache.spark.sql.functions.lit import org.apache.spark.sql.internal.SQLConf.{OPTIMIZER_MAX_ITERATIONS, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.sources._ @@ -198,6 +202,43 @@ class V1WriteFallbackSuite extends QueryTest with SharedSparkSession with Before SparkSession.setDefaultSession(spark) } } + + test("SPARK-50315: metrics for V1 fallback writers") { + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + try { + val session = SparkSession.builder() + .master("local[1]") + .config(V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[V1FallbackTableCatalog].getName) + .getOrCreate() + + def captureWrite(sparkSession: SparkSession)(thunk: => Unit): SparkPlan = { + val physicalPlans = withPhysicalPlansCaptured(sparkSession, thunk) + val v1FallbackWritePlans = physicalPlans.filter { + case _: AppendDataExecV1 | _: OverwriteByExpressionExecV1 => true + case _ => false + } + + assert(v1FallbackWritePlans.size === 1) + v1FallbackWritePlans.head + } + + val appendPlan = captureWrite(session) { + val df = session.createDataFrame(Seq((1, "x"))) + df.write.mode("append").option("name", "t1").format(v2Format).saveAsTable("test") + } + assert(appendPlan.metrics("numOutputRows").value === 1) + + val overwritePlan = captureWrite(session) { + val df2 = session.createDataFrame(Seq((2, "y"))) + df2.writeTo("test").overwrite(lit(true)) + } + assert(overwritePlan.metrics("numOutputRows").value === 1) + } finally { + SparkSession.setActiveSession(spark) + SparkSession.setDefaultSession(spark) + } + } } class V1WriteFallbackSessionCatalogSuite @@ -376,10 +417,23 @@ class InMemoryTableWithV1Fallback( } override def build(): V1Write = new V1Write { + case class SupportedV1WriteMetric(name: String, description: String) extends CustomSumMetric + + override def supportedCustomMetrics(): Array[CustomMetric] = + Array(SupportedV1WriteMetric("numOutputRows", "Number of output rows")) + + private var writeMetrics = Array.empty[CustomTaskMetric] + + override def reportDriverMetrics(): Array[CustomTaskMetric] = writeMetrics + override def toInsertableRelation: InsertableRelation = { (data: DataFrame, overwrite: Boolean) => { assert(!overwrite, "V1 write fallbacks cannot be called with overwrite=true") val rows = data.collect() + + case class V1WriteTaskMetric(name: String, value: Long) extends CustomTaskMetric + writeMetrics = Array(V1WriteTaskMetric("numOutputRows", rows.length)) + rows.groupBy(getPartitionValues).foreach { case (partition, elements) => if (dataMap.contains(partition) && mode == "append") { dataMap.put(partition, dataMap(partition) ++ elements) From 19509d07983d050d0234c03760433bb67c823055 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 19 Nov 2024 21:40:07 +0800 Subject: [PATCH 14/58] Revert "[SPARK-49002][SQL] Consistently handle invalid locations in WAREHOUSE/SCHEMA/TABLE/PARTITION/DIRECTORY" This reverts commit b0f92a9261a34defa016361b1321f634f0516347. --- .../resources/error/error-conditions.json | 6 -- .../catalog/ExternalCatalogUtils.scala | 13 +-- .../sql/errors/QueryExecutionErrors.scala | 10 +-- .../analysis/ResolveSessionCatalog.scala | 10 ++- .../sql/execution/datasources/rules.scala | 14 +-- .../datasources/v2/DataSourceV2Strategy.scala | 8 ++ .../analyzer-results/sql-on-files.sql.out | 80 ----------------- .../sql-tests/inputs/sql-on-files.sql | 5 -- .../sql-tests/results/sql-on-files.sql.out | 88 ------------------- .../AlterNamespaceSetLocationSuiteBase.scala | 15 ---- 10 files changed, 29 insertions(+), 220 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index e8dbac21ab762..58718cb99f551 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2774,12 +2774,6 @@ }, "sqlState" : "42K0E" }, - "INVALID_LOCATION" : { - "message" : [ - "The location name cannot be an invalid URI, but `` was given." - ], - "sqlState" : "42K05" - }, "INVALID_NON_DETERMINISTIC_EXPRESSIONS" : { "message" : [ "The operator expects a deterministic expression, but the actual expression is ." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index ec01fe3ddc37a..8960c7345521c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI -import org.apache.commons.lang3.StringUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.util.Shell @@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BasePredicate, BoundReference, Expression, Predicate} import org.apache.spark.sql.catalyst.expressions.Hex.unhexDigits import org.apache.spark.sql.catalyst.util.CharVarcharUtils -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType @@ -281,15 +280,7 @@ object CatalogUtils { * @return the URI of the path */ def stringToURI(str: String): URI = { - if (StringUtils.isEmpty(str)) { - throw QueryExecutionErrors.invalidLocationError(str, "INVALID_EMPTY_LOCATION") - } - try { - new Path(str).toUri - } catch { - case e: IllegalArgumentException => - throw QueryExecutionErrors.invalidLocationError(str, "INVALID_LOCATION", e) - } + new Path(str).toUri } def makeQualifiedDBObjectPath( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index ba48000f2aeca..ef4fdeeaddfe4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -2538,14 +2538,10 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE ) } - def invalidLocationError( - location: String, - errorClass: String, - cause: Throwable = null): SparkIllegalArgumentException = { + def invalidEmptyLocationError(location: String): SparkIllegalArgumentException = { new SparkIllegalArgumentException( - errorClass = errorClass, - messageParameters = Map("location" -> location), - cause = cause) + errorClass = "INVALID_EMPTY_LOCATION", + messageParameters = Map("location" -> location)) } def malformedProtobufMessageDetectedInMessageParsingError(e: Throwable): Throwable = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 884c870e8eed3..92c74f7bede18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.commons.lang3.StringUtils + import org.apache.spark.SparkException import org.apache.spark.internal.LogKeys.CONFIG import org.apache.spark.internal.MDC @@ -30,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, toPrettySQL, CharVarch import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns._ import org.apache.spark.sql.connector.catalog.{CatalogExtension, CatalogManager, CatalogPlugin, CatalogV2Util, LookupCatalog, SupportsNamespaces, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable => CreateTableV1} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils @@ -137,6 +139,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) AlterDatabasePropertiesCommand(db, properties) case SetNamespaceLocation(ResolvedV1Database(db), location) if conf.useV1Command => + if (StringUtils.isEmpty(location)) { + throw QueryExecutionErrors.invalidEmptyLocationError(location) + } AlterDatabaseSetLocationCommand(db, location) case RenameTable(ResolvedV1TableOrViewIdentifier(oldIdent), newName, isView) => @@ -240,6 +245,9 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) val comment = c.properties.get(SupportsNamespaces.PROP_COMMENT) val location = c.properties.get(SupportsNamespaces.PROP_LOCATION) val newProperties = c.properties -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES + if (location.isDefined && location.get.isEmpty) { + throw QueryExecutionErrors.invalidEmptyLocationError(location.get) + } CreateDatabaseCommand(name, c.ifNotExists, location, comment, newProperties) case d @ DropNamespace(ResolvedV1Database(db), _, _) if conf.useV1Command => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index e1b397aaeb36f..eaad99ab05f8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -22,7 +22,6 @@ import java.util.Locale import scala.collection.mutable.{HashMap, HashSet} import scala.jdk.CollectionConverters._ -import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ @@ -55,7 +54,7 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { val ident = unresolved.multipartIdentifier val dataSource = DataSource( sparkSession, - paths = Seq(CatalogUtils.stringToURI(ident.last).toString), + paths = Seq(ident.last), className = ident.head, options = unresolved.options.asScala.toMap) // `dataSource.providingClass` may throw ClassNotFoundException, the caller side will try-catch @@ -67,6 +66,12 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", messageParameters = Map("dataSourceType" -> ident.head)) } + if (isFileFormat && ident.last.isEmpty) { + unresolved.failAnalysis( + errorClass = "INVALID_EMPTY_LOCATION", + messageParameters = Map("location" -> ident.last)) + } + dataSource } @@ -89,11 +94,6 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { LogicalRelation(ds.resolveRelation()) } catch { case _: ClassNotFoundException => u - case e: SparkIllegalArgumentException if e.getCondition != null => - u.failAnalysis( - errorClass = e.getCondition, - messageParameters = e.getMessageParameters.asScala.toMap, - cause = e) case e: Exception if !e.isInstanceOf[AnalysisException] => // the provider is valid, but failed to create a logical plan u.failAnalysis( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 1c265650e02a7..499721fbae4e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable +import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.Path import org.apache.spark.SparkException @@ -380,6 +381,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil case SetNamespaceLocation(ResolvedNamespace(catalog, ns, _), location) => + if (StringUtils.isEmpty(location)) { + throw QueryExecutionErrors.invalidEmptyLocationError(location) + } AlterNamespaceSetPropertiesExec( catalog.asNamespaceCatalog, ns, @@ -392,6 +396,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat Map(SupportsNamespaces.PROP_COMMENT -> comment)) :: Nil case CreateNamespace(ResolvedNamespace(catalog, ns, _), ifNotExists, properties) => + val location = properties.get(SupportsNamespaces.PROP_LOCATION) + if (location.isDefined && location.get.isEmpty) { + throw QueryExecutionErrors.invalidEmptyLocationError(location.get) + } val finalProperties = properties.get(SupportsNamespaces.PROP_LOCATION).map { loc => properties + (SupportsNamespaces.PROP_LOCATION -> makeQualifiedDBObjectPath(loc)) }.getOrElse(properties) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out index 2899b6c1b0bed..b098a9758fe4e 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out @@ -34,26 +34,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM parquet.`file:tmp` --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 32, - "fragment" : "parquet.`file:tmp`" - } ] -} - - -- !query SELECT * FROM parquet.`/file/not/found` -- !query analysis @@ -109,26 +89,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM orc.`file:tmp` --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 28, - "fragment" : "orc.`file:tmp`" - } ] -} - - -- !query SELECT * FROM orc.`/file/not/found` -- !query analysis @@ -184,26 +144,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM csv.`file:tmp` --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 28, - "fragment" : "csv.`file:tmp`" - } ] -} - - -- !query SELECT * FROM csv.`/file/not/found` -- !query analysis @@ -259,26 +199,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM json.`file:tmp` --- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 29, - "fragment" : "json.`file:tmp`" - } ] -} - - -- !query SELECT * FROM json.`/file/not/found` -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql index 2b45a5060c999..8a00e4400e6b0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql @@ -2,8 +2,6 @@ CREATE DATABASE IF NOT EXISTS sql_on_files; -- Parquet CREATE TABLE sql_on_files.test_parquet USING PARQUET AS SELECT 1; SELECT * FROM parquet.``; -SELECT * FROM parquet.`file:tmp`; - SELECT * FROM parquet.`/file/not/found`; SELECT * FROM parquet.`${spark.sql.warehouse.dir}/sql_on_files.db/test_parquet`; DROP TABLE sql_on_files.test_parquet; @@ -11,7 +9,6 @@ DROP TABLE sql_on_files.test_parquet; -- ORC CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1; SELECT * FROM orc.``; -SELECT * FROM orc.`file:tmp`; SELECT * FROM orc.`/file/not/found`; SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc`; DROP TABLE sql_on_files.test_orc; @@ -19,7 +16,6 @@ DROP TABLE sql_on_files.test_orc; -- CSV CREATE TABLE sql_on_files.test_csv USING CSV AS SELECT 1; SELECT * FROM csv.``; -SELECT * FROM csv.`file:tmp`; SELECT * FROM csv.`/file/not/found`; SELECT * FROM csv.`${spark.sql.warehouse.dir}/sql_on_files.db/test_csv`; DROP TABLE sql_on_files.test_csv; @@ -27,7 +23,6 @@ DROP TABLE sql_on_files.test_csv; -- JSON CREATE TABLE sql_on_files.test_json USING JSON AS SELECT 1; SELECT * FROM json.``; -SELECT * FROM json.`file:tmp`; SELECT * FROM json.`/file/not/found`; SELECT * FROM json.`${spark.sql.warehouse.dir}/sql_on_files.db/test_json`; DROP TABLE sql_on_files.test_json; diff --git a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out index 73eef8fe74f23..fc8f44bc22fee 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out @@ -37,28 +37,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM parquet.`file:tmp` --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 32, - "fragment" : "parquet.`file:tmp`" - } ] -} - - -- !query SELECT * FROM parquet.`/file/not/found` -- !query schema @@ -120,28 +98,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM orc.`file:tmp` --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 28, - "fragment" : "orc.`file:tmp`" - } ] -} - - -- !query SELECT * FROM orc.`/file/not/found` -- !query schema @@ -203,28 +159,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM csv.`file:tmp` --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 28, - "fragment" : "csv.`file:tmp`" - } ] -} - - -- !query SELECT * FROM csv.`/file/not/found` -- !query schema @@ -286,28 +220,6 @@ org.apache.spark.sql.AnalysisException } --- !query -SELECT * FROM json.`file:tmp` --- !query schema -struct<> --- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "INVALID_LOCATION", - "sqlState" : "42K05", - "messageParameters" : { - "location" : "file:tmp" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 15, - "stopIndex" : 29, - "fragment" : "json.`file:tmp`" - } ] -} - - -- !query SELECT * FROM json.`/file/not/found` -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala index 64491f9ad9741..00484c2efc838 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala @@ -56,21 +56,6 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt } } - test("Invalid location string") { - val ns = s"$catalog.$namespace" - withNamespace(ns) { - sql(s"CREATE NAMESPACE $ns") - val sqlText = s"ALTER NAMESPACE $ns SET LOCATION 'file:tmp'" - val e = intercept[SparkIllegalArgumentException] { - sql(sqlText) - } - checkError( - exception = e, - condition = "INVALID_LOCATION", - parameters = Map("location" -> "file:tmp")) - } - } - test("Namespace does not exist") { val ns = "not_exist" val e = intercept[AnalysisException] { From 37497e6ad836787ff7d62805b124602650d9e2c9 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 20 Nov 2024 09:24:57 +0900 Subject: [PATCH 15/58] [SPARK-50335][PYTHON][DOCS][FOLLOW-UP] Make percentile doctests more stable ### What changes were proposed in this pull request? Make percentile doctests more stable: 1, do not compare the numbers exactly; 2, sort the result dataframe of aggregation; ### Why are the changes needed? the test may fail in different envs: cpus, java versions, etc ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48892 from zhengruifeng/stable_doc_tests. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 32 +++++++++++++++++-------- 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5fce4a67f5662..9e4ffc476c14a 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -7138,14 +7138,18 @@ def percentile( >>> key = (sf.col("id") % 3).alias("key") >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select(sf.percentile("value", [0.25, 0.5, 0.75], sf.lit(1))).show(truncate=False) + >>> df.select( + ... sf.percentile("value", [0.25, 0.5, 0.75], sf.lit(1)) + ... ).show(truncate=False) +--------------------------------------------------------+ |percentile(value, array(0.25, 0.5, 0.75), 1) | +--------------------------------------------------------+ |[0.7441991494121..., 9.9900713756..., 19.33740203080...]| +--------------------------------------------------------+ - >>> df.groupBy("key").agg(sf.percentile("value", sf.lit(0.5), sf.lit(1))).show() + >>> df.groupBy("key").agg( + ... sf.percentile("value", sf.lit(0.5), sf.lit(1)) + ... ).sort("key").show() +---+-------------------------+ |key|percentile(value, 0.5, 1)| +---+-------------------------+ @@ -7199,14 +7203,18 @@ def percentile_approx( >>> key = (sf.col("id") % 3).alias("key") >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select(sf.percentile_approx("value", [0.25, 0.5, 0.75], 1000000)).show(truncate=False) + >>> df.select( + ... sf.percentile_approx("value", [0.25, 0.5, 0.75], 1000000) + ... ).show(truncate=False) +----------------------------------------------------------+ |percentile_approx(value, array(0.25, 0.5, 0.75), 1000000) | +----------------------------------------------------------+ |[0.7264430125286..., 9.98975299938..., 19.335304783039...]| +----------------------------------------------------------+ - >>> df.groupBy("key").agg(sf.percentile_approx("value", sf.lit(0.5), sf.lit(1000000))).show() + >>> df.groupBy("key").agg( + ... sf.percentile_approx("value", sf.lit(0.5), sf.lit(1000000)) + ... ).sort("key").show() +---+--------------------------------------+ |key|percentile_approx(value, 0.5, 1000000)| +---+--------------------------------------+ @@ -7256,20 +7264,24 @@ def approx_percentile( >>> key = (sf.col("id") % 3).alias("key") >>> value = (sf.randn(42) + key * 10).alias("value") >>> df = spark.range(0, 1000, 1, 1).select(key, value) - >>> df.select(sf.approx_percentile("value", [0.25, 0.5, 0.75], 1000000)).show(truncate=False) + >>> df.select( + ... sf.approx_percentile("value", [0.25, 0.5, 0.75], 1000000) + ... ).show(truncate=False) +----------------------------------------------------------+ |approx_percentile(value, array(0.25, 0.5, 0.75), 1000000) | +----------------------------------------------------------+ - |[0.7264430125286507, 9.98975299938167, 19.335304783039014]| + |[0.7264430125286..., 9.98975299938..., 19.335304783039...]| +----------------------------------------------------------+ - >>> df.groupBy("key").agg(sf.approx_percentile("value", sf.lit(0.5), sf.lit(1000000))).show() + >>> df.groupBy("key").agg( + ... sf.approx_percentile("value", sf.lit(0.5), sf.lit(1000000)) + ... ).sort("key").show() +---+--------------------------------------+ |key|approx_percentile(value, 0.5, 1000000)| +---+--------------------------------------+ - | 0| -0.03519435193070876| - | 1| 9.990389751837329| - | 2| 19.967859769284075| + | 0| -0.03519435193070...| + | 1| 9.990389751837...| + | 2| 19.967859769284...| +---+--------------------------------------+ """ percentage = lit(list(percentage)) if isinstance(percentage, (list, tuple)) else lit(percentage) From c149dcbc733f3d40ee78a6f01ea59d627733b7de Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 20 Nov 2024 09:25:30 +0900 Subject: [PATCH 16/58] [SPARK-50352][PYTHON][DOCS] Refine docstrings for window/aggregation functions - part 3 ### What changes were proposed in this pull request? Refine docstrings for window/aggregation functions: 1, make examples copy-pasteable; 2, specify the meaning of str input: either column name or literal string; 3, show both input and output columns; 4, add see also section for related functions; ### Why are the changes needed? 1, improve the documentation; 2, improve the doctest coverage; ### Does this PR introduce _any_ user-facing change? doc changes ### How was this patch tested? updated doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48891 from zhengruifeng/py_doc_5. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 461 +++++++++++++++--------- 1 file changed, 300 insertions(+), 161 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 9e4ffc476c14a..23bec6024c94e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -1514,6 +1514,10 @@ def count(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` column for computed results. + See Also + -------- + :meth:`pyspark.sql.functions.count_if` + Examples -------- Example 1: Count all rows in a DataFrame @@ -1736,6 +1740,10 @@ def median(col: "ColumnOrName") -> Column: ----- Supports Spark Connect. + :meth:`pyspark.sql.functions.percentile` + :meth:`pyspark.sql.functions.approx_percentile` + :meth:`pyspark.sql.functions.percentile_approx` + Examples -------- >>> from pyspark.sql import functions as sf @@ -6881,6 +6889,11 @@ def input_file_name() -> Column: :class:`~pyspark.sql.Column` file names. + See Also + -------- + :meth:`pyspark.sql.functions.input_file_block_length` + :meth:`pyspark.sql.functions.input_file_block_start` + Examples -------- >>> import os @@ -7132,6 +7145,12 @@ def percentile( :class:`~pyspark.sql.Column` the exact `percentile` of the numeric column. + See Also + -------- + :meth:`pyspark.sql.functions.median` + :meth:`pyspark.sql.functions.approx_percentile` + :meth:`pyspark.sql.functions.percentile_approx` + Examples -------- >>> from pyspark.sql import functions as sf @@ -7197,6 +7216,12 @@ def percentile_approx( :class:`~pyspark.sql.Column` approximate `percentile` of the numeric column. + See Also + -------- + :meth:`pyspark.sql.functions.median` + :meth:`pyspark.sql.functions.percentile` + :meth:`pyspark.sql.functions.approx_percentile` + Examples -------- >>> from pyspark.sql import functions as sf @@ -7258,6 +7283,12 @@ def approx_percentile( :class:`~pyspark.sql.Column` approximate `percentile` of the numeric column. + See Also + -------- + :meth:`pyspark.sql.functions.median` + :meth:`pyspark.sql.functions.percentile` + :meth:`pyspark.sql.functions.percentile_approx` + Examples -------- >>> from pyspark.sql import functions as sf @@ -7768,7 +7799,7 @@ def struct( Parameters ---------- - cols : list, set, str or :class:`~pyspark.sql.Column` + cols : list, set, :class:`~pyspark.sql.Column` or column name column names or :class:`~pyspark.sql.Column`\\s to contain in the output struct. Returns @@ -7776,13 +7807,21 @@ def struct( :class:`~pyspark.sql.Column` a struct type column of given columns. + See Also + -------- + :meth:`pyspark.sql.functions.named_struct` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([("Alice", 2), ("Bob", 5)], ("name", "age")) - >>> df.select(struct('age', 'name').alias("struct")).collect() - [Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))] - >>> df.select(struct([df.age, df.name]).alias("struct")).collect() - [Row(struct=Row(age=2, name='Alice')), Row(struct=Row(age=5, name='Bob'))] + >>> df.select("*", sf.struct('age', df.name)).show() + +-----+---+-----------------+ + | name|age|struct(age, name)| + +-----+---+-----------------+ + |Alice| 2| {2, Alice}| + | Bob| 5| {5, Bob}| + +-----+---+-----------------+ """ if len(cols) == 1 and isinstance(cols[0], (list, set)): cols = cols[0] # type: ignore[assignment] @@ -7798,18 +7837,27 @@ def named_struct(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name list of columns to work on. Returns ------- :class:`~pyspark.sql.Column` + See Also + -------- + :meth:`pyspark.sql.functions.struct` + Examples -------- - >>> df = spark.createDataFrame([(1, 2, 3)], ['a', 'b', 'c']) - >>> df.select(named_struct(lit('x'), df.a, lit('y'), df.b).alias('r')).collect() - [Row(r=Row(x=1, y=2))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(1, 2)], ['a', 'b']) + >>> df.select("*", sf.named_struct(sf.lit('x'), df.a, sf.lit('y'), "b")).show() + +---+---+------------------------+ + | a| b|named_struct(x, a, y, b)| + +---+---+------------------------+ + | 1| 2| {1, 2}| + +---+---+------------------------+ """ return _invoke_function_over_seq_of_columns("named_struct", cols) @@ -7827,7 +7875,7 @@ def greatest(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols: :class:`~pyspark.sql.Column` or str + cols: :class:`~pyspark.sql.Column` or column name columns to check for greatest value. Returns @@ -7835,11 +7883,20 @@ def greatest(*cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` greatest value. + See Also + -------- + :meth:`pyspark.sql.functions.least` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) - >>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect() - [Row(greatest=4)] + >>> df.select("*", sf.greatest(df.a, "b", df.c)).show() + +---+---+---+-----------------+ + | a| b| c|greatest(a, b, c)| + +---+---+---+-----------------+ + | 1| 4| 3| 4| + +---+---+---+-----------------+ """ if len(cols) < 2: raise PySparkValueError( @@ -7862,7 +7919,7 @@ def least(*cols: "ColumnOrName") -> Column: Parameters ---------- - cols : :class:`~pyspark.sql.Column` or str + cols : :class:`~pyspark.sql.Column` or column name column names or columns to be compared Returns @@ -7870,11 +7927,20 @@ def least(*cols: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` least value. + See Also + -------- + :meth:`pyspark.sql.functions.greatest` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c']) - >>> df.select(least(df.a, df.b, df.c).alias("least")).collect() - [Row(least=1)] + >>> df.select("*", sf.least(df.a, "b", df.c)).show() + +---+---+---+--------------+ + | a| b| c|least(a, b, c)| + +---+---+---+--------------+ + | 1| 4| 3| 1| + +---+---+---+--------------+ """ if len(cols) < 2: raise PySparkValueError( @@ -7907,26 +7973,32 @@ def when(condition: Column, value: Any) -> Column: :class:`~pyspark.sql.Column` column representing when expression. + See Also + -------- + :meth:`pyspark.sql.Column.when` + :meth:`pyspark.sql.Column.otherwise` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.range(3) - >>> df.select(when(df['id'] == 2, 3).otherwise(4).alias("age")).show() - +---+ - |age| - +---+ - | 4| - | 4| - | 3| - +---+ - - >>> df.select(when(df.id == 2, df.id + 1).alias("age")).show() - +----+ - | age| - +----+ - |NULL| - |NULL| - | 3| - +----+ + >>> df.select("*", sf.when(df['id'] == 2, 3).otherwise(4)).show() + +---+------------------------------------+ + | id|CASE WHEN (id = 2) THEN 3 ELSE 4 END| + +---+------------------------------------+ + | 0| 4| + | 1| 4| + | 2| 3| + +---+------------------------------------+ + + >>> df.select("*", sf.when(df.id == 2, df.id + 1)).show() + +---+------------------------------------+ + | id|CASE WHEN (id = 2) THEN (id + 1) END| + +---+------------------------------------+ + | 0| NULL| + | 1| NULL| + | 2| 3| + +---+------------------------------------+ """ # Explicitly not using ColumnOrName type here to make reading condition less opaque if not isinstance(condition, Column): @@ -8208,7 +8280,7 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name name of column or expression offset : int, optional default 1 number of row to extend @@ -8220,14 +8292,16 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> :class:`~pyspark.sql.Column` value before current row based on `offset`. + See Also + -------- + :meth:`pyspark.sql.functions.lead` + Examples -------- + >>> from pyspark.sql import functions as sf >>> from pyspark.sql import Window - >>> df = spark.createDataFrame([("a", 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) + >>> df = spark.createDataFrame( + ... [("a", 1), ("a", 2), ("a", 3), ("b", 8), ("b", 2)], ["c1", "c2"]) >>> df.show() +---+---+ | c1| c2| @@ -8238,8 +8312,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 8| | b| 2| +---+---+ + >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("previous_value", lag("c2").over(w)).show() + >>> df.withColumn("previous_value", sf.lag("c2").over(w)).show() +---+---+--------------+ | c1| c2|previous_value| +---+---+--------------+ @@ -8249,7 +8324,8 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| NULL| | b| 8| 2| +---+---+--------------+ - >>> df.withColumn("previous_value", lag("c2", 1, 0).over(w)).show() + + >>> df.withColumn("previous_value", sf.lag("c2", 1, 0).over(w)).show() +---+---+--------------+ | c1| c2|previous_value| +---+---+--------------+ @@ -8259,7 +8335,8 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| 0| | b| 8| 2| +---+---+--------------+ - >>> df.withColumn("previous_value", lag("c2", 2, -1).over(w)).show() + + >>> df.withColumn("previous_value", sf.lag("c2", 2, -1).over(w)).show() +---+---+--------------+ | c1| c2|previous_value| +---+---+--------------+ @@ -8293,7 +8370,7 @@ def lead(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name name of column or expression offset : int, optional default 1 number of row to extend @@ -8305,14 +8382,16 @@ def lead(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> :class:`~pyspark.sql.Column` value after current row based on `offset`. + See Also + -------- + :meth:`pyspark.sql.functions.lag` + Examples -------- + >>> from pyspark.sql import functions as sf >>> from pyspark.sql import Window - >>> df = spark.createDataFrame([("a", 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) + >>> df = spark.createDataFrame( + ... [("a", 1), ("a", 2), ("a", 3), ("b", 8), ("b", 2)], ["c1", "c2"]) >>> df.show() +---+---+ | c1| c2| @@ -8323,8 +8402,9 @@ def lead(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 8| | b| 2| +---+---+ + >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("next_value", lead("c2").over(w)).show() + >>> df.withColumn("next_value", sf.lead("c2").over(w)).show() +---+---+----------+ | c1| c2|next_value| +---+---+----------+ @@ -8334,7 +8414,8 @@ def lead(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| 8| | b| 8| NULL| +---+---+----------+ - >>> df.withColumn("next_value", lead("c2", 1, 0).over(w)).show() + + >>> df.withColumn("next_value", sf.lead("c2", 1, 0).over(w)).show() +---+---+----------+ | c1| c2|next_value| +---+---+----------+ @@ -8344,7 +8425,8 @@ def lead(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| 8| | b| 8| 0| +---+---+----------+ - >>> df.withColumn("next_value", lead("c2", 2, -1).over(w)).show() + + >>> df.withColumn("next_value", sf.lead("c2", 2, -1).over(w)).show() +---+---+----------+ | c1| c2|next_value| +---+---+----------+ @@ -8380,7 +8462,7 @@ def nth_value(col: "ColumnOrName", offset: int, ignoreNulls: Optional[bool] = Fa Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name name of column or expression offset : int number of row to use as the value @@ -8395,12 +8477,10 @@ def nth_value(col: "ColumnOrName", offset: int, ignoreNulls: Optional[bool] = Fa Examples -------- + >>> from pyspark.sql import functions as sf >>> from pyspark.sql import Window - >>> df = spark.createDataFrame([("a", 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) + >>> df = spark.createDataFrame( + ... [("a", 1), ("a", 2), ("a", 3), ("b", 8), ("b", 2)], ["c1", "c2"]) >>> df.show() +---+---+ | c1| c2| @@ -8411,8 +8491,9 @@ def nth_value(col: "ColumnOrName", offset: int, ignoreNulls: Optional[bool] = Fa | b| 8| | b| 2| +---+---+ + >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("nth_value", nth_value("c2", 1).over(w)).show() + >>> df.withColumn("nth_value", sf.nth_value("c2", 1).over(w)).show() +---+---+---------+ | c1| c2|nth_value| +---+---+---------+ @@ -8422,7 +8503,8 @@ def nth_value(col: "ColumnOrName", offset: int, ignoreNulls: Optional[bool] = Fa | b| 2| 2| | b| 8| 2| +---+---+---------+ - >>> df.withColumn("nth_value", nth_value("c2", 2).over(w)).show() + + >>> df.withColumn("nth_value", sf.nth_value("c2", 2).over(w)).show() +---+---+---------+ | c1| c2|nth_value| +---+---+---------+ @@ -8448,7 +8530,7 @@ def any_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] = Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. ignoreNulls : :class:`~pyspark.sql.Column` or bool, optional if first value is null then look for first non-null value. @@ -8460,15 +8542,22 @@ def any_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] = Examples -------- - >>> df = spark.createDataFrame([(None, 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) - >>> df.select(any_value('c1'), any_value('c2')).collect() - [Row(any_value(c1)=None, any_value(c2)=1)] - >>> df.select(any_value('c1', True), any_value('c2', True)).collect() - [Row(any_value(c1)='a', any_value(c2)=1)] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame( + ... [(None, 1), ("a", 2), ("a", 3), ("b", 8), ("b", 2)], ["c1", "c2"]) + >>> df.select(sf.any_value('c1'), sf.any_value('c2')).show() + +-------------+-------------+ + |any_value(c1)|any_value(c2)| + +-------------+-------------+ + | NULL| 1| + +-------------+-------------+ + + >>> df.select(sf.any_value('c1', True), sf.any_value('c2', True)).show() + +-------------+-------------+ + |any_value(c1)|any_value(c2)| + +-------------+-------------+ + | a| 1| + +-------------+-------------+ """ if ignoreNulls is None: return _invoke_function_over_columns("any_value", col) @@ -8489,7 +8578,7 @@ def first_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. ignoreNulls : :class:`~pyspark.sql.Column` or bool, optional if first value is null then look for first non-null value. @@ -8499,6 +8588,10 @@ def first_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] :class:`~pyspark.sql.Column` some value of `col` for a group of rows. + See Also + -------- + :meth:`pyspark.sql.functions.last_value` + Examples -------- >>> import pyspark.sql.functions as sf @@ -8540,7 +8633,7 @@ def last_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] = Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. ignoreNulls : :class:`~pyspark.sql.Column` or bool, optional if first value is null then look for first non-null value. @@ -8550,6 +8643,10 @@ def last_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] = :class:`~pyspark.sql.Column` some value of `col` for a group of rows. + See Also + -------- + :meth:`pyspark.sql.functions.first_value` + Examples -------- >>> import pyspark.sql.functions as sf @@ -8591,7 +8688,7 @@ def count_if(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name target column to work on. Returns @@ -8599,6 +8696,10 @@ def count_if(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the number of `TRUE` values for the `col`. + See Also + -------- + :meth:`pyspark.sql.functions.count` + Examples -------- Example 1: Counting the number of even numbers in a numeric column @@ -8638,13 +8739,13 @@ def count_if(col: "ColumnOrName") -> Column: Example 4: Counting the number of rows where a boolean column is True >>> from pyspark.sql import functions as sf - >>> df = spark.createDataFrame([(True,), (False,), (True,), (False,), (True,)], ["bool"]) - >>> df.select(sf.count_if(sf.col('bool'))).show() - +--------------+ - |count_if(bool)| - +--------------+ - | 3| - +--------------+ + >>> df = spark.createDataFrame([(True,), (False,), (True,), (False,), (True,)], ["b"]) + >>> df.select(sf.count('b'), sf.count_if('b')).show() + +--------+-----------+ + |count(b)|count_if(b)| + +--------+-----------+ + | 5| 3| + +--------+-----------+ """ return _invoke_function_over_columns("count_if", col) @@ -8717,12 +8818,10 @@ def ntile(n: int) -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> from pyspark.sql import Window - >>> df = spark.createDataFrame([("a", 1), - ... ("a", 2), - ... ("a", 3), - ... ("b", 8), - ... ("b", 2)], ["c1", "c2"]) + >>> df = spark.createDataFrame( + ... [("a", 1), ("a", 2), ("a", 3), ("b", 8), ("b", 2)], ["c1", "c2"]) >>> df.show() +---+---+ | c1| c2| @@ -8733,8 +8832,9 @@ def ntile(n: int) -> Column: | b| 8| | b| 2| +---+---+ + >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("ntile", ntile(2).over(w)).show() + >>> df.withColumn("ntile", sf.ntile(2).over(w)).show() +---+---+-----+ | c1| c2|ntile| +---+---+-----+ @@ -23030,7 +23130,7 @@ def hll_sketch_agg( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name lgConfigK : :class:`~pyspark.sql.Column` or int, optional The log-base-2 of K, where K is the number of buckets or slots for the HllSketch @@ -23039,33 +23139,29 @@ def hll_sketch_agg( :class:`~pyspark.sql.Column` The binary representation of the HllSketch. + See Also + -------- + :meth:`pyspark.sql.functions.hll_union` + :meth:`pyspark.sql.functions.hll_union_agg` + :meth:`pyspark.sql.functions.hll_sketch_estimate` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([1,2,2,3], "INT") - >>> df1 = df.agg(hll_sketch_estimate(hll_sketch_agg("value")).alias("distinct_cnt")) - >>> df1.show() - +------------+ - |distinct_cnt| - +------------+ - | 3| - +------------+ - >>> df2 = df.agg(hll_sketch_estimate( - ... hll_sketch_agg("value", lit(12)) - ... ).alias("distinct_cnt")) - >>> df2.show() - +------------+ - |distinct_cnt| - +------------+ - | 3| - +------------+ - >>> df3 = df.agg(hll_sketch_estimate( - ... hll_sketch_agg(col("value"), lit(12))).alias("distinct_cnt")) - >>> df3.show() - +------------+ - |distinct_cnt| - +------------+ - | 3| - +------------+ + >>> df.agg(sf.hll_sketch_estimate(sf.hll_sketch_agg("value"))).show() + +----------------------------------------------+ + |hll_sketch_estimate(hll_sketch_agg(value, 12))| + +----------------------------------------------+ + | 3| + +----------------------------------------------+ + + >>> df.agg(sf.hll_sketch_estimate(sf.hll_sketch_agg("value", 12))).show() + +----------------------------------------------+ + |hll_sketch_estimate(hll_sketch_agg(value, 12))| + +----------------------------------------------+ + | 3| + +----------------------------------------------+ """ if lgConfigK is None: return _invoke_function_over_columns("hll_sketch_agg", col) @@ -23088,7 +23184,7 @@ def hll_union_agg( Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name allowDifferentLgConfigK : :class:`~pyspark.sql.Column` or bool, optional Allow sketches with different lgConfigK values to be merged (defaults to false). @@ -23097,39 +23193,33 @@ def hll_union_agg( :class:`~pyspark.sql.Column` The binary representation of the merged HllSketch. + See Also + -------- + :meth:`pyspark.sql.functions.hll_union` + :meth:`pyspark.sql.functions.hll_sketch_agg` + :meth:`pyspark.sql.functions.hll_sketch_estimate` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df1 = spark.createDataFrame([1,2,2,3], "INT") - >>> df1 = df1.agg(hll_sketch_agg("value").alias("sketch")) + >>> df1 = df1.agg(sf.hll_sketch_agg("value").alias("sketch")) >>> df2 = spark.createDataFrame([4,5,5,6], "INT") - >>> df2 = df2.agg(hll_sketch_agg("value").alias("sketch")) - >>> df3 = df1.union(df2).agg(hll_sketch_estimate( - ... hll_union_agg("sketch") - ... ).alias("distinct_cnt")) - >>> df3.drop("sketch").show() - +------------+ - |distinct_cnt| - +------------+ - | 6| - +------------+ - >>> df4 = df1.union(df2).agg(hll_sketch_estimate( - ... hll_union_agg("sketch", lit(False)) - ... ).alias("distinct_cnt")) - >>> df4.drop("sketch").show() - +------------+ - |distinct_cnt| - +------------+ - | 6| - +------------+ - >>> df5 = df1.union(df2).agg(hll_sketch_estimate( - ... hll_union_agg(col("sketch"), lit(False)) - ... ).alias("distinct_cnt")) - >>> df5.drop("sketch").show() - +------------+ - |distinct_cnt| - +------------+ - | 6| - +------------+ + >>> df2 = df2.agg(sf.hll_sketch_agg("value").alias("sketch")) + >>> df3 = df1.union(df2) + >>> df3.agg(sf.hll_sketch_estimate(sf.hll_union_agg("sketch"))).show() + +-------------------------------------------------+ + |hll_sketch_estimate(hll_union_agg(sketch, false))| + +-------------------------------------------------+ + | 6| + +-------------------------------------------------+ + + >>> df3.agg(sf.hll_sketch_estimate(sf.hll_union_agg("sketch", False))).show() + +-------------------------------------------------+ + |hll_sketch_estimate(hll_union_agg(sketch, false))| + +-------------------------------------------------+ + | 6| + +-------------------------------------------------+ """ if allowDifferentLgConfigK is None: return _invoke_function_over_columns("hll_union_agg", col) @@ -23147,23 +23237,29 @@ def hll_sketch_estimate(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name Returns ------- :class:`~pyspark.sql.Column` The estimated number of unique values for the HllSketch. + See Also + -------- + :meth:`pyspark.sql.functions.hll_union` + :meth:`pyspark.sql.functions.hll_union_agg` + :meth:`pyspark.sql.functions.hll_sketch_agg` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([1,2,2,3], "INT") - >>> df = df.agg(hll_sketch_estimate(hll_sketch_agg("value")).alias("distinct_cnt")) - >>> df.show() - +------------+ - |distinct_cnt| - +------------+ - | 3| - +------------+ + >>> df.agg(sf.hll_sketch_estimate(sf.hll_sketch_agg("value"))).show() + +----------------------------------------------+ + |hll_sketch_estimate(hll_sketch_agg(value, 12))| + +----------------------------------------------+ + | 3| + +----------------------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -23183,8 +23279,8 @@ def hll_union( Parameters ---------- - col1 : :class:`~pyspark.sql.Column` or str - col2 : :class:`~pyspark.sql.Column` or str + col1 : :class:`~pyspark.sql.Column` or column name + col2 : :class:`~pyspark.sql.Column` or column name allowDifferentLgConfigK : bool, optional Allow sketches with different lgConfigK values to be merged (defaults to false). @@ -23193,17 +23289,26 @@ def hll_union( :class:`~pyspark.sql.Column` The binary representation of the merged HllSketch. + See Also + -------- + :meth:`pyspark.sql.functions.hll_union_agg` + :meth:`pyspark.sql.functions.hll_sketch_agg` + :meth:`pyspark.sql.functions.hll_sketch_estimate` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(1,4),(2,5),(2,5),(3,6)], "struct") - >>> df = df.agg(hll_sketch_agg("v1").alias("sketch1"), hll_sketch_agg("v2").alias("sketch2")) - >>> df = df.withColumn("distinct_cnt", hll_sketch_estimate(hll_union("sketch1", "sketch2"))) - >>> df.drop("sketch1", "sketch2").show() - +------------+ - |distinct_cnt| - +------------+ - | 6| - +------------+ + >>> df = df.agg( + ... sf.hll_sketch_agg("v1").alias("sketch1"), + ... sf.hll_sketch_agg("v2").alias("sketch2") + ... ) + >>> df.select(sf.hll_sketch_estimate(sf.hll_union(df.sketch1, "sketch2"))).show() + +-------------------------------------------------------+ + |hll_sketch_estimate(hll_union(sketch1, sketch2, false))| + +-------------------------------------------------------+ + | 6| + +-------------------------------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -23827,11 +23932,28 @@ def input_file_block_length() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.input_file_name` + :meth:`pyspark.sql.functions.input_file_block_start` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.read.text("python/test_support/sql/ages_newlines.csv", lineSep=",") - >>> df.select(input_file_block_length().alias('r')).first() - Row(r=87) + >>> df.select(sf.input_file_block_length()).show() + +-------------------------+ + |input_file_block_length()| + +-------------------------+ + | 87| + | 87| + | 87| + | 87| + | 87| + | 87| + | 87| + | 87| + +-------------------------+ """ return _invoke_function_over_columns("input_file_block_length") @@ -23843,11 +23965,28 @@ def input_file_block_start() -> Column: .. versionadded:: 3.5.0 + See Also + -------- + :meth:`pyspark.sql.functions.input_file_name` + :meth:`pyspark.sql.functions.input_file_block_length` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.read.text("python/test_support/sql/ages_newlines.csv", lineSep=",") - >>> df.select(input_file_block_start().alias('r')).first() - Row(r=0) + >>> df.select(sf.input_file_block_start()).show() + +------------------------+ + |input_file_block_start()| + +------------------------+ + | 0| + | 0| + | 0| + | 0| + | 0| + | 0| + | 0| + | 0| + +------------------------+ """ return _invoke_function_over_columns("input_file_block_start") From 87917675c4013b4f979c576bc7586efaa3d69a41 Mon Sep 17 00:00:00 2001 From: Milan Dankovic Date: Wed, 20 Nov 2024 10:07:50 +0800 Subject: [PATCH 17/58] [SPARK-48344][SQL] Prepare SQL Scripting for addition of Execution Framework ### What changes were proposed in this pull request? This PR is Initial refactoring of SQL Scripting to prepare it for addition of **Execution Framework**: - Move all files to proper directories/paths. - Convert `SqlScriptingLogicalOperators` to `SqlScriptingLogicalPlans`. - Remove `CompoundNestedStatementIteratorExec` because it is unnecessary abstraction. - Remove `parseScript` because it is no more needed. Parsing is done in `parsePlan` method. ### Why are the changes needed? This changes are needed so execution of SQL Scripts can be implemented properly. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48879 from miland-db/milan-dankovic_data/refactor-execution-1. Authored-by: Milan Dankovic Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 5 + .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../spark/sql/catalyst/parser/parsers.scala | 14 + .../catalyst/parser/AbstractSqlParser.scala | 20 +- .../sql/catalyst/parser/AstBuilder.scala | 11 +- .../sql/catalyst/parser/ParserInterface.scala | 6 - .../logical/SqlScriptingLogicalPlans.scala} | 141 ++++++- .../spark/sql/errors/SqlScriptingErrors.scala | 11 + .../apache/spark/sql/internal/SQLConf.scala | 9 + .../parser/SqlScriptingParserSuite.scala | 362 ++++++++++-------- ...parkConnectWithSessionExtensionSuite.scala | 5 +- .../scripting/SqlScriptingExecutionNode.scala | 29 +- .../scripting/SqlScriptingInterpreter.scala | 3 +- .../sql/SparkSessionExtensionSuite.scala | 5 +- .../SqlScriptingInterpreterSuite.scala | 48 ++- 15 files changed, 431 insertions(+), 240 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/{parser/SqlScriptingLogicalOperators.scala => plans/logical/SqlScriptingLogicalPlans.scala} (58%) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 58718cb99f551..dfa22d5ec125e 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5370,6 +5370,11 @@ " is a VARIABLE and cannot be updated using the SET statement. Use SET VARIABLE = ... instead." ] }, + "SQL_SCRIPTING" : { + "message" : [ + "SQL Scripting is under development and not all features are supported. SQL Scripting enables users to write procedural SQL including control flow and error handling. To enable existing features set to `true`." + ] + }, "STATE_STORE_MULTIPLE_COLUMN_FAMILIES" : { "message" : [ "Creating multiple column families with is not supported." diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 0d049f4b18e0e..55a4b85ecb6bf 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -48,7 +48,7 @@ compoundOrSingleStatement ; singleCompoundStatement - : beginEndCompoundBlock SEMICOLON? EOF + : BEGIN compoundBody END SEMICOLON? EOF ; beginEndCompoundBlock diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index f2c7dd533af3a..54af195847dac 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -415,6 +415,20 @@ case class UnclosedCommentProcessor(command: String, tokenStream: CommonTokenStr } } + override def exitCompoundOrSingleStatement( + ctx: SqlBaseParser.CompoundOrSingleStatementContext): Unit = { + // Same as in exitSingleStatement, we shouldn't parse the comments in SET command. + if (Option(ctx.singleStatement()).forall( + !_.setResetStatement().isInstanceOf[SqlBaseParser.SetConfigurationContext])) { + checkUnclosedComment(tokenStream, command) + } + } + + override def exitSingleCompoundStatement( + ctx: SqlBaseParser.SingleCompoundStatementContext): Unit = { + checkUnclosedComment(tokenStream, command) + } + /** check `has_unclosed_bracketed_comment` to find out the unclosed bracketed comment. */ private def checkUnclosedComment(tokenStream: CommonTokenStream, command: String) = { assert(tokenStream.getTokenSource.isInstanceOf[SqlBaseLexer]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala index 1c477964a6890..1928d4b23349b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala @@ -21,7 +21,7 @@ import org.antlr.v4.runtime.ParserRuleContext import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.ParserUtils.withOrigin -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{CompoundPlanStatement, LogicalPlan} import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.errors.QueryParsingErrors @@ -80,9 +80,10 @@ abstract class AbstractSqlParser extends AbstractParser with ParserInterface { /** Creates LogicalPlan for a given SQL string. */ override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser => - val ctx = parser.singleStatement() + val ctx = parser.compoundOrSingleStatement() withErrorHandling(ctx, Some(sqlText)) { - astBuilder.visitSingleStatement(ctx) match { + astBuilder.visitCompoundOrSingleStatement(ctx) match { + case compoundBody: CompoundPlanStatement => compoundBody case plan: LogicalPlan => plan case _ => val position = Origin(None, None) @@ -91,19 +92,6 @@ abstract class AbstractSqlParser extends AbstractParser with ParserInterface { } } - /** Creates [[CompoundBody]] for a given SQL script string. */ - override def parseScript(sqlScriptText: String): CompoundBody = parse(sqlScriptText) { parser => - val ctx = parser.compoundOrSingleStatement() - withErrorHandling(ctx, Some(sqlScriptText)) { - astBuilder.visitCompoundOrSingleStatement(ctx) match { - case body: CompoundBody => body - case _ => - val position = Origin(None, None) - throw QueryParsingErrors.sqlStatementUnsupportedError(sqlScriptText, position) - } - } - } - def withErrorHandling[T](ctx: ParserRuleContext, sqlText: Option[String])(toResult: => T): T = { withOrigin(ctx, sqlText) { try { 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 add1112247ea7..1604a0b5aba19 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 @@ -131,19 +131,20 @@ class AstBuilder extends DataTypeAstBuilder } override def visitCompoundOrSingleStatement( - ctx: CompoundOrSingleStatementContext): CompoundBody = withOrigin(ctx) { + ctx: CompoundOrSingleStatementContext): LogicalPlan = withOrigin(ctx) { Option(ctx.singleCompoundStatement()).map { s => + if (!conf.getConf(SQLConf.SQL_SCRIPTING_ENABLED)) { + throw SqlScriptingErrors.sqlScriptingNotEnabled(CurrentOrigin.get) + } visit(s).asInstanceOf[CompoundBody] }.getOrElse { - val logicalPlan = visitSingleStatement(ctx.singleStatement()) - CompoundBody(Seq(SingleStatement(parsedPlan = logicalPlan)), - Some(java.util.UUID.randomUUID.toString.toLowerCase(Locale.ROOT))) + visitSingleStatement(ctx.singleStatement()) } } override def visitSingleCompoundStatement(ctx: SingleCompoundStatementContext): CompoundBody = { val labelCtx = new SqlScriptingLabelContext() - visitBeginEndCompoundBlockImpl(ctx.beginEndCompoundBlock(), labelCtx) + visitCompoundBodyImpl(ctx.compoundBody(), None, allowVarDeclare = true, labelCtx) } private def visitCompoundBodyImpl( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala index 04edb0f75c463..3aec1dd431138 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala @@ -62,10 +62,4 @@ trait ParserInterface extends DataTypeParserInterface { */ @throws[ParseException]("Text cannot be parsed to a LogicalPlan") def parseQuery(sqlText: String): LogicalPlan - - /** - * Parse a SQL script string to a [[CompoundBody]]. - */ - @throws[ParseException]("Text cannot be parsed to a CompoundBody") - def parseScript(sqlScriptText: String): CompoundBody } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala similarity index 58% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala index 9fd87f51bd57e..e6018e5e57b9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SqlScriptingLogicalPlans.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.parser +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, WithOrigin} /** * Trait for all SQL Scripting logical operators that are product of parsing phase. * These operators will be used by the SQL Scripting interpreter to generate execution nodes. */ -sealed trait CompoundPlanStatement +sealed trait CompoundPlanStatement extends LogicalPlan /** * Logical operator representing result of parsing a single SQL statement @@ -32,8 +33,7 @@ sealed trait CompoundPlanStatement * @param parsedPlan Result of SQL statement parsing. */ case class SingleStatement(parsedPlan: LogicalPlan) - extends CompoundPlanStatement - with WithOrigin { + extends CompoundPlanStatement { override val origin: Origin = CurrentOrigin.get @@ -46,6 +46,14 @@ case class SingleStatement(parsedPlan: LogicalPlan) assert(origin.sqlText.isDefined && origin.startIndex.isDefined && origin.stopIndex.isDefined) origin.sqlText.get.substring(origin.startIndex.get, origin.stopIndex.get + 1) } + + override def output: Seq[Attribute] = parsedPlan.output + + override def children: Seq[LogicalPlan] = parsedPlan.children + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = + SingleStatement(parsedPlan.withNewChildren(newChildren)) } /** @@ -57,7 +65,15 @@ case class SingleStatement(parsedPlan: LogicalPlan) */ case class CompoundBody( collection: Seq[CompoundPlanStatement], - label: Option[String]) extends CompoundPlanStatement + label: Option[String]) extends Command with CompoundPlanStatement { + + override def children: Seq[LogicalPlan] = collection + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + CompoundBody(newChildren.map(_.asInstanceOf[CompoundPlanStatement]), label) + } +} /** * Logical operator for IF ELSE statement. @@ -73,6 +89,30 @@ case class IfElseStatement( conditionalBodies: Seq[CompoundBody], elseBody: Option[CompoundBody]) extends CompoundPlanStatement { assert(conditions.length == conditionalBodies.length) + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.concat(conditions, conditionalBodies, elseBody) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + val conditions = newChildren + .filter(_.isInstanceOf[SingleStatement]) + .map(_.asInstanceOf[SingleStatement]) + var conditionalBodies = newChildren + .filter(_.isInstanceOf[CompoundBody]) + .map(_.asInstanceOf[CompoundBody]) + var elseBody: Option[CompoundBody] = None + + assert(conditions.length == conditionalBodies.length || + conditions.length + 1 == conditionalBodies.length) + + if (conditions.length < conditionalBodies.length) { + conditionalBodies = conditionalBodies.dropRight(1) + elseBody = Some(conditionalBodies.last) + } + IfElseStatement(conditions, conditionalBodies, elseBody) + } } /** @@ -88,7 +128,21 @@ case class IfElseStatement( case class WhileStatement( condition: SingleStatement, body: CompoundBody, - label: Option[String]) extends CompoundPlanStatement + label: Option[String]) extends CompoundPlanStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq(condition, body) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(newChildren.length == 2) + WhileStatement( + newChildren(0).asInstanceOf[SingleStatement], + newChildren(1).asInstanceOf[CompoundBody], + label) + } +} /** * Logical operator for REPEAT statement. @@ -104,7 +158,21 @@ case class WhileStatement( case class RepeatStatement( condition: SingleStatement, body: CompoundBody, - label: Option[String]) extends CompoundPlanStatement + label: Option[String]) extends CompoundPlanStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq(condition, body) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(newChildren.length == 2) + RepeatStatement( + newChildren(0).asInstanceOf[SingleStatement], + newChildren(1).asInstanceOf[CompoundBody], + label) + } +} /** * Logical operator for LEAVE statement. @@ -113,7 +181,14 @@ case class RepeatStatement( * with the next statement after the body/loop. * @param label Label of the compound or loop to leave. */ -case class LeaveStatement(label: String) extends CompoundPlanStatement +case class LeaveStatement(label: String) extends CompoundPlanStatement { + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = LeaveStatement(label) +} /** * Logical operator for ITERATE statement. @@ -122,7 +197,14 @@ case class LeaveStatement(label: String) extends CompoundPlanStatement * with the next iteration. * @param label Label of the loop to iterate. */ -case class IterateStatement(label: String) extends CompoundPlanStatement +case class IterateStatement(label: String) extends CompoundPlanStatement { + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = IterateStatement(label) +} /** * Logical operator for CASE statement. @@ -136,6 +218,30 @@ case class CaseStatement( conditionalBodies: Seq[CompoundBody], elseBody: Option[CompoundBody]) extends CompoundPlanStatement { assert(conditions.length == conditionalBodies.length) + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.concat(conditions, conditionalBodies, elseBody) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + val conditions = newChildren + .filter(_.isInstanceOf[SingleStatement]) + .map(_.asInstanceOf[SingleStatement]) + var conditionalBodies = newChildren + .filter(_.isInstanceOf[CompoundBody]) + .map(_.asInstanceOf[CompoundBody]) + var elseBody: Option[CompoundBody] = None + + assert(conditions.length == conditionalBodies.length || + conditions.length + 1 == conditionalBodies.length) + + if (conditions.length < conditionalBodies.length) { + conditionalBodies = conditionalBodies.dropRight(1) + elseBody = Some(conditionalBodies.last) + } + CaseStatement(conditions, conditionalBodies, elseBody) + } } /** @@ -149,4 +255,15 @@ case class CaseStatement( */ case class LoopStatement( body: CompoundBody, - label: Option[String]) extends CompoundPlanStatement + label: Option[String]) extends CompoundPlanStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq(body) + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(newChildren.length == 1) + LoopStatement(newChildren(0).asInstanceOf[CompoundBody], label) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala index f94ff1beb458a..f1c07200d503b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.errors import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.catalyst.util.QuotingUtils.toSQLConf import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryExecutionErrors.toSQLStmt import org.apache.spark.sql.exceptions.SqlScriptingException +import org.apache.spark.sql.internal.SQLConf /** * Object for grouping error messages thrown during parsing/interpreting phase @@ -82,6 +84,15 @@ private[sql] object SqlScriptingErrors { messageParameters = Map("invalidStatement" -> toSQLStmt(stmt))) } + def sqlScriptingNotEnabled(origin: Origin): Throwable = { + new SqlScriptingException( + errorClass = "UNSUPPORTED_FEATURE.SQL_SCRIPTING", + cause = null, + origin = origin, + messageParameters = Map( + "sqlScriptingEnabled" -> toSQLConf(SQLConf.SQL_SCRIPTING_ENABLED.key))) + } + def booleanStatementWithEmptyRow( origin: Origin, stmt: String): Throwable = { 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 5218a683a8fa8..0123f91393e7e 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 @@ -3423,6 +3423,15 @@ object SQLConf { .version("2.3.0") .fallbackConf(org.apache.spark.internal.config.STRING_REDACTION_PATTERN) + val SQL_SCRIPTING_ENABLED = + buildConf("spark.sql.scripting.enabled") + .doc("SQL Scripting feature is under development and its use should be done under this " + + "feature flag. SQL Scripting enables users to write procedural SQL including control " + + "flow and error handling.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + val CONCAT_BINARY_AS_STRING = buildConf("spark.sql.function.concatBinaryAsString") .doc("When this option is set to false and all inputs are binary, `functions.concat` returns " + "an output as binary. Otherwise, it returns as a string.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 9a77f87f87946..3bb84f603dc67 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -20,35 +20,46 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions.{Alias, EqualTo, Expression, In, Literal, ScalarSubquery} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, Project} +import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CreateVariable, IfElseStatement, IterateStatement, LeaveStatement, LoopStatement, Project, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.exceptions.SqlScriptingException +import org.apache.spark.sql.internal.SQLConf class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { import CatalystSqlParser._ - test("single select") { - val sqlScriptText = "SELECT 1;" - val tree = parseScript(sqlScriptText) - assert(tree.collection.length == 1) - assert(tree.collection.head.isInstanceOf[SingleStatement]) - val sparkStatement = tree.collection.head.asInstanceOf[SingleStatement] - assert(sparkStatement.getText == "SELECT 1;") + // Tests setup + protected override def beforeAll(): Unit = { + super.beforeAll() + conf.setConfString(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") } - test("single select without ;") { - val sqlScriptText = "SELECT 1" - val tree = parseScript(sqlScriptText) - assert(tree.collection.length == 1) - assert(tree.collection.head.isInstanceOf[SingleStatement]) - val sparkStatement = tree.collection.head.asInstanceOf[SingleStatement] - assert(sparkStatement.getText == "SELECT 1") + protected override def afterAll(): Unit = { + conf.unsetConf(SQLConf.SQL_SCRIPTING_ENABLED.key) + super.afterAll() + } + + // Tests + test("single select") { + val sqlScriptText = "SELECT 1;" + val statement = parsePlan(sqlScriptText) + assert(!statement.isInstanceOf[CompoundBody]) } test("multi select without ; - should fail") { val sqlScriptText = "SELECT 1 SELECT 1" val e = intercept[ParseException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText) + } + assert(e.getCondition === "PARSE_SYNTAX_ERROR") + assert(e.getMessage.contains("Syntax error")) + assert(e.getMessage.contains("SELECT")) + } + + test("multi select with ; - should fail") { + val sqlScriptText = "SELECT 1; SELECT 1;" + val e = intercept[ParseException] { + parsePlan(sqlScriptText) } assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) @@ -57,7 +68,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("multi select") { val sqlScriptText = "BEGIN SELECT 1;SELECT 2; END" - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) @@ -76,7 +87,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { """ |BEGIN |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.isEmpty) } @@ -88,7 +99,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT 2; |END""".stripMargin val e = intercept[ParseException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) @@ -103,7 +114,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT 2 |END""".stripMargin val e = intercept[ParseException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) @@ -120,7 +131,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SELECT a, b, c FROM T; | SELECT * FROM T; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 5) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) sqlScriptText.split(";") @@ -147,7 +158,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END; | END; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[CompoundBody]) val body1 = tree.collection.head.asInstanceOf[CompoundBody] @@ -165,17 +176,37 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { == "SELECT 3") } - test("compound: beginLabel") { + // TODO: to be removed once the parser rule for top level compound is fixed to support labels! + test("top level compound: labels not allowed") { val sqlScriptText = """ |lbl: BEGIN | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; |END""".stripMargin - val tree = parseScript(sqlScriptText) + checkError( + exception = intercept[ParseException] { + parsePlan(sqlScriptText) + }, + condition = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'lbl'", "hint" -> "")) + } + + test("compound: beginLabel") { + val sqlScriptText = + """ + |BEGIN + | lbl: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END; + |END""".stripMargin + val rootTree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(rootTree.collection.length == 1) + + val tree = rootTree.collection.head.asInstanceOf[CompoundBody] assert(tree.collection.length == 5) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) assert(tree.label.contains("lbl")) @@ -184,14 +215,19 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("compound: beginLabel + endLabel") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; - |END lbl""".stripMargin - val tree = parseScript(sqlScriptText) + |BEGIN + | lbl: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END lbl; + |END""".stripMargin + val rootTree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(rootTree.collection.length == 1) + + val tree = rootTree.collection.head.asInstanceOf[CompoundBody] assert(tree.collection.length == 5) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) assert(tree.label.contains("lbl")) @@ -200,70 +236,84 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("compound: beginLabel + endLabel with different values") { val sqlScriptText = """ - |lbl_begin: BEGIN - | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; - |END lbl_end""".stripMargin + |BEGIN + | lbl_begin: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END lbl_end; + |END""".stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText) } checkError( exception = exception, condition = "LABELS_MISMATCH", parameters = Map("beginLabel" -> toSQLId("lbl_begin"), "endLabel" -> toSQLId("lbl_end"))) - assert(exception.origin.line.contains(2)) + assert(exception.origin.line.contains(3)) } test("compound: endLabel") { val sqlScriptText = """ |BEGIN - | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; - |END lbl""".stripMargin + | BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END lbl; + |END""".stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText) } checkError( exception = exception, condition = "END_LABEL_WITHOUT_BEGIN_LABEL", parameters = Map("endLabel" -> toSQLId("lbl"))) - assert(exception.origin.line.contains(8)) + assert(exception.origin.line.contains(9)) } test("compound: beginLabel + endLabel with different casing") { val sqlScriptText = """ - |LBL: BEGIN - | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; - |END lbl""".stripMargin - val tree = parseScript(sqlScriptText) + |BEGIN + | LBL: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END lbl; + |END""".stripMargin + val rootTree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(rootTree.collection.length == 1) + + val tree = rootTree.collection.head.asInstanceOf[CompoundBody] assert(tree.collection.length == 5) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) assert(tree.label.contains("lbl")) } - test("compound: no labels provided") { + test("compound: no labels provided, random label should be generated") { val sqlScriptText = """ |BEGIN - | SELECT 1; - | SELECT 2; - | INSERT INTO A VALUES (a, b, 3); - | SELECT a, b, c FROM T; - | SELECT * FROM T; + | BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + | END; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val rootTree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(rootTree.collection.length == 1) + + val tree = rootTree.collection.head.asInstanceOf[CompoundBody] assert(tree.collection.length == 5) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) assert(tree.label.nonEmpty) @@ -276,7 +326,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE testVariable1 VARCHAR(50); | DECLARE testVariable2 INTEGER; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) assert(tree.collection.forall( @@ -291,12 +341,12 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE testVariable INTEGER; |END""".stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText) } checkError( - exception = exception, - condition = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", - parameters = Map("varName" -> "`testVariable`")) + exception = exception, + condition = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING", + parameters = Map("varName" -> "`testVariable`")) assert(exception.origin.line.contains(4)) } @@ -309,7 +359,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END IF; |END""".stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText) } checkError( exception = exception, @@ -325,7 +375,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE totalInsCnt = 0; | SET VAR totalInsCnt = (SELECT x FROM y WHERE id = 1); |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[SingleStatement]) assert(tree.collection(1).isInstanceOf[SingleStatement]) @@ -338,7 +388,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE totalInsCnt = 0; | SET VARIABLE totalInsCnt = (SELECT x FROM y WHERE id = 1); |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[SingleStatement]) assert(tree.collection(1).isInstanceOf[SingleStatement]) @@ -351,7 +401,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | DECLARE totalInsCnt = 0; | SET totalInsCnt = (SELECT x FROM y WHERE id = 1); |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[SingleStatement]) assert(tree.collection(1).isInstanceOf[SingleStatement]) @@ -365,7 +415,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | SET totalInsCnt = (SELECT x FROMERROR y WHERE id = 1); |END""".stripMargin val e = intercept[ParseException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) @@ -380,7 +430,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END IF; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[IfElseStatement]) val ifStmt = tree.collection.head.asInstanceOf[IfElseStatement] @@ -399,7 +449,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END IF; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[IfElseStatement]) @@ -434,7 +484,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END IF; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[IfElseStatement]) @@ -474,7 +524,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END IF; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[IfElseStatement]) @@ -518,7 +568,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END IF; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[IfElseStatement]) @@ -557,7 +607,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END WHILE lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -575,16 +625,15 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("while with complex condition") { val sqlScriptText = - """ - |BEGIN - |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; - |WHILE (SELECT COUNT(*) < 2 FROM t) DO - | SELECT 42; - |END WHILE; - |END - |""".stripMargin - - val tree = parseScript(sqlScriptText) + """ + |BEGIN + |CREATE TABLE t (a INT, b STRING, c DOUBLE) USING parquet; + |WHILE (SELECT COUNT(*) < 2 FROM t) DO + | SELECT 42; + |END WHILE; + |END + |""".stripMargin + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection(1).isInstanceOf[WhileStatement]) @@ -610,7 +659,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END WHILE lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -651,7 +700,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END WHILE lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -679,11 +728,16 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("leave compound block") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | LEAVE lbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | LEAVE lbl; + | END; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val rootTree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] + assert(rootTree.collection.length == 1) + + val tree = rootTree.collection.head.asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[SingleStatement]) assert(tree.collection(1).isInstanceOf[LeaveStatement]) @@ -698,7 +752,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | LEAVE lbl; | END WHILE; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -726,7 +780,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | UNTIL 1 = 2 | END REPEAT; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -747,13 +801,15 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test ("iterate compound block - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | ITERATE lbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | ITERATE lbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] }, condition = "INVALID_LABEL_USAGE.ITERATE_IN_COMPOUND", parameters = Map("labelName" -> "LBL")) @@ -768,7 +824,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | ITERATE lbl; | END WHILE; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -796,7 +852,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | UNTIL 1 = 2 | END REPEAT; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -817,13 +873,15 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("leave with wrong label - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | LEAVE randomlbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | LEAVE randomlbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] }, condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "LEAVE")) @@ -832,13 +890,15 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { test("iterate with wrong label - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | ITERATE randomlbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | ITERATE randomlbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] }, condition = "INVALID_LABEL_USAGE.DOES_NOT_EXIST", parameters = Map("labelName" -> "RANDOMLBL", "statementType" -> "ITERATE")) @@ -855,7 +915,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END WHILE; | END WHILE; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -890,7 +950,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | UNTIL 1 = 1 | END REPEAT; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -924,7 +984,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END WHILE; | END WHILE; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[WhileStatement]) @@ -959,7 +1019,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | UNTIL 1 = 1 | END REPEAT; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -991,7 +1051,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END REPEAT lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -1020,7 +1080,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection(1).isInstanceOf[RepeatStatement]) @@ -1048,7 +1108,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END REPEAT lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -1093,7 +1153,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END REPEAT lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[RepeatStatement]) @@ -1129,7 +1189,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] @@ -1152,7 +1212,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) @@ -1196,7 +1256,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] @@ -1225,7 +1285,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) @@ -1267,7 +1327,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] @@ -1291,7 +1351,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) @@ -1310,7 +1370,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { assert(caseStmt.conditions(1).isInstanceOf[SingleStatement]) checkSimpleCaseStatementCondition( - caseStmt.conditions(1), _ == Literal(1), _.isInstanceOf[ScalarSubquery]) + caseStmt.conditions(1), _ == Literal(1), _.isInstanceOf[ScalarSubquery]) assert(caseStmt.conditionalBodies(1).collection.head.isInstanceOf[SingleStatement]) assert(caseStmt.conditionalBodies(1).collection.head.asInstanceOf[SingleStatement] @@ -1337,7 +1397,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) val caseStmt = tree.collection.head.asInstanceOf[CaseStatement] @@ -1366,7 +1426,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END CASE; |END |""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CaseStatement]) @@ -1409,7 +1469,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END LOOP lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1436,7 +1496,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1475,7 +1535,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END LOOP lbl; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1504,7 +1564,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | LEAVE lbl; | END LOOP; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1529,7 +1589,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | ITERATE lbl; | END LOOP; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1556,7 +1616,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END LOOP; | END LOOP; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1586,7 +1646,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { | END LOOP; | END LOOP; |END""".stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[LoopStatement]) @@ -1616,7 +1676,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1634,7 +1694,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CompoundBody]) val body_1 = tree.collection.head.asInstanceOf[CompoundBody] @@ -1658,7 +1718,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1677,7 +1737,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1696,7 +1756,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1715,7 +1775,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1736,7 +1796,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1755,7 +1815,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END """.stripMargin val exception = intercept[SqlScriptingException] { - parseScript(sqlScriptText) + parsePlan(sqlScriptText).asInstanceOf[CompoundBody] } checkError( exception = exception, @@ -1774,7 +1834,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 2) assert(tree.collection.head.isInstanceOf[CompoundBody]) assert(tree.collection.head.asInstanceOf[CompoundBody].label.get == "lbl") @@ -1800,7 +1860,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END REPEAT; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 4) assert(tree.collection.head.isInstanceOf[CompoundBody]) assert(tree.collection.head.asInstanceOf[CompoundBody].label.get == "lbl") @@ -1827,7 +1887,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { |END; |END """.stripMargin - val tree = parseScript(sqlScriptText) + val tree = parsePlan(sqlScriptText).asInstanceOf[CompoundBody] assert(tree.collection.length == 1) assert(tree.collection.head.isInstanceOf[CompoundBody]) // Compound body diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala index 46d86b8bb6ee5..e8b955cf33ebc 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface} +import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connect.SparkConnectTestUtils import org.apache.spark.sql.types.{DataType, StructType} @@ -54,9 +54,6 @@ class SparkConnectWithSessionExtensionSuite extends SparkFunSuite { override def parseQuery(sqlText: String): LogicalPlan = delegate.parseQuery(sqlText) - - override def parseScript(sqlScriptText: String): CompoundBody = - delegate.parseScript(sqlScriptText) } test("Parse table name with test parser") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index 3f8b0f1583adb..9129fc6ab00f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -136,19 +136,16 @@ class SingleStatementExec( } /** - * Abstract class for all statements that contain nested statements. - * Implements recursive iterator logic over all child execution nodes. - * @param collection - * Collection of child execution nodes. + * Executable node for CompoundBody. + * @param statements + * Executable nodes for nested statements within the CompoundBody. * @param label - * Label set by user or None otherwise. + * Label set by user to CompoundBody or None otherwise. */ -abstract class CompoundNestedStatementIteratorExec( - collection: Seq[CompoundStatementExec], - label: Option[String] = None) +class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[String] = None) extends NonLeafStatementExec { - private var localIterator = collection.iterator + private var localIterator = statements.iterator private var curr = if (localIterator.hasNext) Some(localIterator.next()) else None /** Used to stop the iteration in cases when LEAVE statement is encountered. */ @@ -207,8 +204,8 @@ abstract class CompoundNestedStatementIteratorExec( override def getTreeIterator: Iterator[CompoundStatementExec] = treeIterator override def reset(): Unit = { - collection.foreach(_.reset()) - localIterator = collection.iterator + statements.foreach(_.reset()) + localIterator = statements.iterator curr = if (localIterator.hasNext) Some(localIterator.next()) else None stopIteration = false } @@ -244,16 +241,6 @@ abstract class CompoundNestedStatementIteratorExec( } } -/** - * Executable node for CompoundBody. - * @param statements - * Executable nodes for nested statements within the CompoundBody. - * @param label - * Label set by user to CompoundBody or None otherwise. - */ -class CompoundBodyExec(statements: Seq[CompoundStatementExec], label: Option[String] = None) - extends CompoundNestedStatementIteratorExec(statements, label) - /** * Executable node for IfElseStatement. * @param conditions Collection of executable conditions. First condition corresponds to IF clause, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala index 78ef715e18982..1be75cb61c8b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreter.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.scripting import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedIdentifier -import org.apache.spark.sql.catalyst.parser.{CaseStatement, CompoundBody, CompoundPlanStatement, IfElseStatement, IterateStatement, LeaveStatement, LoopStatement, RepeatStatement, SingleStatement, WhileStatement} -import org.apache.spark.sql.catalyst.plans.logical.{CreateVariable, DropVariable, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{CaseStatement, CompoundBody, CompoundPlanStatement, CreateVariable, DropVariable, IfElseStatement, IterateStatement, LeaveStatement, LogicalPlan, LoopStatement, RepeatStatement, SingleStatement, WhileStatement} import org.apache.spark.sql.catalyst.trees.Origin /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 8750c398cc942..90cca58b3bd29 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, Max, Partial} -import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, CompoundBody, ParserInterface} +import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.{PlanTest, SQLHelper} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, AggregateHint, ColumnStat, Limit, LocalRelation, LogicalPlan, Sort, SortHint, Statistics, UnresolvedHint} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} @@ -608,9 +608,6 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars override def parseQuery(sqlText: String): LogicalPlan = delegate.parseQuery(sqlText) - - override def parseScript(sqlScriptText: String): CompoundBody = - delegate.parseScript(sqlScriptText) } object MyExtensions { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala index 3551608a1ee84..b0b844d2b52ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/scripting/SqlScriptingInterpreterSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.scripting -import org.apache.spark.{SparkException, SparkNumberFormatException} +import org.apache.spark.{SparkConf, SparkException, SparkNumberFormatException} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row} import org.apache.spark.sql.catalyst.QueryPlanningTracker +import org.apache.spark.sql.catalyst.plans.logical.CompoundBody import org.apache.spark.sql.exceptions.SqlScriptingException import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -31,10 +32,16 @@ import org.apache.spark.sql.test.SharedSparkSession * are executed and output DataFrames are compared with expected outputs. */ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { + + // Tests setup + override protected def sparkConf: SparkConf = { + super.sparkConf.set(SQLConf.SQL_SCRIPTING_ENABLED.key, "true") + } + // Helpers private def runSqlScript(sqlText: String): Array[DataFrame] = { val interpreter = SqlScriptingInterpreter() - val compoundBody = spark.sessionState.sqlParser.parseScript(sqlText) + val compoundBody = spark.sessionState.sqlParser.parsePlan(sqlText).asInstanceOf[CompoundBody] val executionPlan = interpreter.buildExecutionPlan(compoundBody, spark) executionPlan.flatMap { case statement: SingleStatementExec => @@ -54,10 +61,6 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { } // Tests - test("select 1") { - verifySqlScriptResult("SELECT 1;", Seq(Seq(Row(1)))) - } - test("multi statement - simple") { withTable("t") { val sqlScript = @@ -1121,9 +1124,12 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { test("leave compound block") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | LEAVE lbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | LEAVE lbl; + | SELECT 2; + | END; |END""".stripMargin val expected = Seq( Seq(Row(1)) // select @@ -1165,9 +1171,11 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { test("iterate compound block - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | ITERATE lbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | ITERATE lbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { @@ -1229,9 +1237,11 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { test("leave with wrong label - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | LEAVE randomlbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | LEAVE randomlbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { @@ -1244,9 +1254,11 @@ class SqlScriptingInterpreterSuite extends QueryTest with SharedSparkSession { test("iterate with wrong label - should fail") { val sqlScriptText = """ - |lbl: BEGIN - | SELECT 1; - | ITERATE randomlbl; + |BEGIN + | lbl: BEGIN + | SELECT 1; + | ITERATE randomlbl; + | END; |END""".stripMargin checkError( exception = intercept[SqlScriptingException] { From b7cf448aa4099d38cdba22764cbd142d33cee6fb Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 20 Nov 2024 10:39:58 +0800 Subject: [PATCH 18/58] [SPARK-49550][FOLLOWUP][SQL][DOC] Switch Hadoop to 3.4.1 in IsolatedClientLoader and docs ### What changes were proposed in this pull request? Switch Hadoop to 3.4.1 in `IsolatedClientLoader` and docs. ### Why are the changes needed? Make the Hadoop version consistent in the code and docs. ### Does this PR introduce _any_ user-facing change? Docs are updated. ### How was this patch tested? Pass GHA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48890 from pan3793/SPARK-49550-followup. Authored-by: Cheng Pan Signed-off-by: yangjie01 --- assembly/README | 2 +- docs/building-spark.md | 7 ++----- docs/running-on-kubernetes.md | 2 +- docs/running-on-yarn.md | 2 +- pom.xml | 3 +-- .../spark/sql/hive/client/IsolatedClientLoader.scala | 2 +- 6 files changed, 7 insertions(+), 11 deletions(-) diff --git a/assembly/README b/assembly/README index ad1305c5b4d56..10c8254ae153e 100644 --- a/assembly/README +++ b/assembly/README @@ -9,4 +9,4 @@ This module is off by default. To activate it specify the profile in the command If you need to build an assembly for a different version of Hadoop the hadoop-version system property needs to be set as in this example: - -Dhadoop.version=3.4.0 + -Dhadoop.version=3.4.1 diff --git a/docs/building-spark.md b/docs/building-spark.md index 547add0fc9f48..518d936c3c85c 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -72,14 +72,11 @@ This will build Spark distribution along with Python pip and R packages. For mor ## Specifying the Hadoop Version and Enabling YARN -You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. - -You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different -from `hadoop.version`. +You can enable the `yarn` profile and specify the exact version of Hadoop to compile against through the `hadoop.version` property. Example: - ./build/mvn -Pyarn -Dhadoop.version=3.4.0 -DskipTests clean package + ./build/mvn -Pyarn -Dhadoop.version=3.4.1 -DskipTests clean package ## Building With Hive and JDBC Support diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 7d6126edaa061..a0c73813612d0 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -236,7 +236,7 @@ A typical example of this using S3 is via passing the following options: ``` ... ---packages org.apache.hadoop:hadoop-aws:3.4.0 +--packages org.apache.hadoop:hadoop-aws:3.4.1 --conf spark.kubernetes.file.upload.path=s3a:///path --conf spark.hadoop.fs.s3a.access.key=... --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b8e22b12e3c92..d149f9196b345 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -33,7 +33,7 @@ Please see [Spark Security](security.html) and the specific security sections in # Launching Spark on YARN -Apache Hadoop does not support Java 17 as of 3.4.0, while Apache Spark requires at least Java 17 since 4.0.0, so a different JDK should be configured for Spark applications. +Apache Hadoop does not support Java 17 as of 3.4.1, while Apache Spark requires at least Java 17 since 4.0.0, so a different JDK should be configured for Spark applications. Please refer to [Configuring different JDKs for Spark Applications](#configuring-different-jdks-for-spark-applications) for details. Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. diff --git a/pom.xml b/pom.xml index 363f8d9453790..c641aea53373d 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,6 @@ 4.28.3 3.11.4 - ${hadoop.version} 3.9.3 5.7.1 org.apache.hive @@ -1418,7 +1417,7 @@ org.apache.hadoop hadoop-client-minicluster - ${yarn.version} + ${hadoop.version} test diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 5d6f928d53dad..482983e698a25 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -66,7 +66,7 @@ private[hive] object IsolatedClientLoader extends Logging { case e: RuntimeException if e.getMessage.contains("hadoop") => // If the error message contains hadoop, it is probably because the hadoop // version cannot be resolved. - val fallbackVersion = "3.4.0" + val fallbackVersion = "3.4.1" logWarning(log"Failed to resolve Hadoop artifacts for the version " + log"${MDC(HADOOP_VERSION, hadoopVersion)}. We will change the hadoop version from " + log"${MDC(HADOOP_VERSION, hadoopVersion)} to " + From 2185f3cbb328e18a905e6822b7ec71ba25fcabca Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 20 Nov 2024 11:59:47 +0900 Subject: [PATCH 19/58] [SPARK-50359][PYTHON] Upgrade PyArrow to 18.0 ### What changes were proposed in this pull request? Upgrade PyArrow to 18.0 ### Why are the changes needed? to test against the latest pyarrow ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48898 from zhengruifeng/infra_pyarrow_18. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- dev/infra/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 26f8f5b5dd8a7..2817818cbc4e3 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -24,7 +24,7 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241028 +ENV FULL_REFRESH_DATE 20241119 ENV DEBIAN_FRONTEND noninteractive ENV DEBCONF_NONINTERACTIVE_SEEN true @@ -94,7 +94,7 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matplotlib lxml -ARG BASIC_PIP_PKGS="numpy pyarrow>=15.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=18.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0 graphviz==0.20.3" From 0157778a3d4be628516c4a50c4fc7f70b0c920d6 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 20 Nov 2024 12:06:50 +0900 Subject: [PATCH 20/58] [SPARK-50358][SQL][TESTS] Update postgres docker image to 17.1 ### What changes were proposed in this pull request? This PR aims to update the `postgres` docker image from `17.0` to `17.1`. ### Why are the changes needed? This will help Apache Spark test the latest postgres. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48897 from panbingkun/SPARK-50358. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/jdbc/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala | 6 +++--- .../spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala | 6 +++--- .../apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala | 6 +++--- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 071b976f044c3..92a3e99586b5f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -32,9 +32,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.0-alpine): + * To run this test suite for a specific version (e.g., postgres:17.1-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.PostgresIntegrationSuite" * }}} @@ -42,7 +42,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index fe121dfc7c324..7c9fc477dbb78 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.datasources.jdbc.connection.SecureConnecti import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.0-alpine): + * To run this test suite for a specific version (e.g., postgres:17.1-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *PostgresKrbIntegrationSuite" * }}} @@ -38,7 +38,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected val keytabFileName = "postgres.keytab" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala index a79bbf39a71b8..b6917df2d428a 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.tags.DockerTest /** * This suite is used to generate subqueries, and test Spark against Postgres. - * To run this test suite for a specific version (e.g., postgres:17.0-alpine): + * To run this test suite for a specific version (e.g., postgres:17.1-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.GeneratedSubquerySuite" * }}} @@ -39,7 +39,7 @@ import org.apache.spark.tags.DockerTest class GeneratedSubquerySuite extends DockerJDBCIntegrationSuite with QueryGeneratorHelper { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala index 80ba35df6c893..56a83cc0a34d6 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/PostgreSQLQueryTestSuite.scala @@ -30,9 +30,9 @@ import org.apache.spark.tags.DockerTest * confidence, and you won't have to manually verify the golden files generated with your test. * 2. Add this line to your .sql file: --ONLY_IF spark * - * Note: To run this test suite for a specific version (e.g., postgres:17.0-alpine): + * Note: To run this test suite for a specific version (e.g., postgres:17.1-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests * "testOnly org.apache.spark.sql.jdbc.PostgreSQLQueryTestSuite" * }}} @@ -45,7 +45,7 @@ class PostgreSQLQueryTestSuite extends CrossDbmsQueryTestSuite { protected val customInputFilePath: String = new File(inputFilePath, "subquery").getAbsolutePath override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index 83b665853c535..95465cc6e40c5 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.0-alpine) + * To run this test suite for a specific version (e.g., postgres:17.1-alpine) * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresIntegrationSuite" * }}} */ @@ -38,7 +38,7 @@ import org.apache.spark.tags.DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { override val catalogName: String = "postgresql" override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala index 6d4f1cc2fd3fc..75f7ede5bc733 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresNamespaceSuite.scala @@ -26,16 +26,16 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., postgres:17.0-alpine): + * To run this test suite for a specific version (e.g., postgres:17.1-alpine): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.0-alpine + * ENABLE_DOCKER_INTEGRATION_TESTS=1 POSTGRES_DOCKER_IMAGE_NAME=postgres:17.1-alpine * ./build/sbt -Pdocker-integration-tests "testOnly *v2.PostgresNamespaceSuite" * }}} */ @DockerTest class PostgresNamespaceSuite extends DockerJDBCIntegrationSuite with V2JDBCNamespaceTest { override val db = new DatabaseOnDocker { - override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.0-alpine") + override val imageName = sys.env.getOrElse("POSTGRES_DOCKER_IMAGE_NAME", "postgres:17.1-alpine") override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) From b582daca568f7098c7b8fe4e3068e5986acdc18b Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 20 Nov 2024 11:41:52 +0800 Subject: [PATCH 21/58] [MINOR][DOCS] Fix a HTML/Markdown syntax error in sql-migration-guide.md ### What changes were proposed in this pull request? This PR fixes the below HTML/Markdown syntax error in sql-migration-guide.md ![image](https://github.com/user-attachments/assets/bb62a240-1ee5-4763-92c2-97fdd5436284) ### Why are the changes needed? docfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? ![image](https://github.com/user-attachments/assets/95b83aa0-beb1-418c-be08-02310010f4d8) ### Was this patch authored or co-authored using generative AI tooling? no Closes #48899 from yaooqinn/minor. Authored-by: Kent Yao Signed-off-by: Kent Yao --- docs/sql-migration-guide.md | 239 ++++++++++++++++-------------------- 1 file changed, 104 insertions(+), 135 deletions(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 0ecd45c2d8c56..ea4dbe926d146 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -637,142 +637,111 @@ license: | - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
- InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
- NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
- IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
- LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
- DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
- DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
- DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
- TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
- StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
InputA \ InputBNullTypeIntegerTypeLongTypeDecimalType(38,0)*DoubleTypeDateTypeTimestampTypeStringType
NullTypeNullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
IntegerTypeIntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
LongTypeLongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
DecimalType(38,0)*DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
DoubleTypeDoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
DateTypeDateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
TimestampTypeTimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. From 19b8250588bf5a4be39a53c4c60fcb5f85a8ec1e Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 20 Nov 2024 12:40:16 +0800 Subject: [PATCH 22/58] [SPARK-50331][INFRA][FOLLOW-UP] Skip Torch/DeepSpeed tests in MacOS PySpark Daily test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Skip Torch/DeepSpeed tests in MacOS PySpark Daily test https://github.com/apache/spark/actions/runs/11921746968/job/33226552068 ### Why are the changes needed? we don't need to test them on MacOS ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? They should be skipped due to no installation: https://github.com/apache/spark/blob/b724955bb7c2995fbbd9c7fe550e44f16397cb5b/python/pyspark/ml/torch/tests/test_data_loader.py#L36 manually test in my local MacOS: ``` (spark_312) ➜ spark git:(master) python/run-tests -k --python-executables python3 --testnames 'pyspark.ml.torch.tests.test_data_loader' Running PySpark tests. Output is in /Users/ruifeng.zheng/Dev/spark/python/unit-tests.log Will test against the following Python executables: ['python3'] Will test the following Python tests: ['pyspark.ml.torch.tests.test_data_loader'] python3 python_implementation is CPython python3 version is: Python 3.12.7 Starting test(python3): pyspark.ml.torch.tests.test_data_loader (temp output: /Users/ruifeng.zheng/Dev/spark/python/target/c7bf947a-a746-4519-b475-aed24a1f8cec/python3__pyspark.ml.torch.tests.test_data_loader__1yq21_l5.log) Finished test(python3): pyspark.ml.torch.tests.test_data_loader (0s) ... 1 tests were skipped Tests passed in 0 seconds Skipped tests in pyspark.ml.torch.tests.test_data_loader with python3: test_data_loader (pyspark.ml.torch.tests.test_data_loader.TorchDistributorDataLoaderUnitTests.test_data_loader) ... skip (0.001s) ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48900 from zhengruifeng/macos_py_lib. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/python_macos_test.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.github/workflows/python_macos_test.yml b/.github/workflows/python_macos_test.yml index 32a8e21801db3..cca133dab541a 100644 --- a/.github/workflows/python_macos_test.yml +++ b/.github/workflows/python_macos_test.yml @@ -133,11 +133,8 @@ jobs: run: | python${{matrix.python}} -m pip install --ignore-installed 'blinker>=1.6.2' python${{matrix.python}} -m pip install --ignore-installed 'six==1.16.0' - python${{matrix.python}} -m pip install py-cpuinfo && \ python${{matrix.python}} -m pip install numpy 'pyarrow>=15.0.0' 'six==1.16.0' 'pandas==2.2.3' scipy 'plotly>=4.8' 'mlflow>=2.8.1' coverage matplotlib openpyxl 'memory-profiler>=0.61.0' 'scikit-learn>=1.3.2' unittest-xml-reporting && \ python${{matrix.python}} -m pip install 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.28.3' 'googleapis-common-protos==1.65.0' 'graphviz==0.20.3' && \ - python${{matrix.python}} -m pip install torch torchvision --index-url https://download.pytorch.org/whl/cpu && \ - python${{matrix.python}} -m pip install deepspeed torcheval && \ python${{matrix.python}} -m pip cache purge && \ python${{matrix.python}} -m pip list # Run the tests. From 7a4f3c4c493828db55409a75fcedfa4bc22045b1 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Wed, 20 Nov 2024 13:39:46 +0800 Subject: [PATCH 23/58] [SPARK-50345][BUILD] Upgrade Kafka to 3.9.0 ### What changes were proposed in this pull request? The pr aims to upgrade `kafka` from `3.8.1` to `3.9.0`. ### Why are the changes needed? https://downloads.apache.org/kafka/3.9.0/RELEASE_NOTES.html ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48885 from panbingkun/SPARK-50345. Authored-by: panbingkun Signed-off-by: Kent Yao --- .../scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala | 4 ++-- pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index c3f02eebab23a..9e06b6c6ff4a2 100644 --- a/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/connector/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -43,7 +43,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.security.auth.SecurityProtocol.{PLAINTEXT, SASL_PLAINTEXT} import org.apache.kafka.common.serialization.StringSerializer -import org.apache.kafka.common.utils.SystemTime +import org.apache.kafka.common.utils.Time import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.apache.zookeeper.server.auth.SASLAuthenticationProvider @@ -253,7 +253,7 @@ class KafkaTestUtils( // Get the actual zookeeper binding port zkPort = zookeeper.actualPort zkClient = KafkaZkClient(s"$zkHost:$zkPort", isSecure = false, zkSessionTimeout, - zkConnectionTimeout, 1, new SystemTime(), "test", new ZKClientConfig) + zkConnectionTimeout, 1, Time.SYSTEM, "test", new ZKClientConfig) zkReady = true } diff --git a/pom.xml b/pom.xml index c641aea53373d..05c2566ac5f30 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ 2.3.10 - 3.8.1 + 3.9.0 10.16.1.1 1.14.4 From 3151d9793521361d29db9a3fc517082843a2629a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 20 Nov 2024 15:37:10 +0900 Subject: [PATCH 24/58] [SPARK-49801][INFRA][FOLLOWUP] Sync pandas version in release environment ### What changes were proposed in this pull request? A follow-up of SPARK-49801 to sync the pandas version in the release environment ### Why are the changes needed? prevent the document generation phase of the release process from import error ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Passing existing CI ### Was this patch authored or co-authored using generative AI tooling? no Closes #48902 from yaooqinn/SPARK-49801. Authored-by: Kent Yao Signed-off-by: Hyukjin Kwon --- dev/create-release/spark-rm/Dockerfile | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index fd7c3dbaa61d6..8a9890bf68dde 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -97,10 +97,10 @@ RUN mkdir -p /usr/local/pypy/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3.10 && \ ln -sf /usr/local/pypy/pypy3.10/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 -RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.2' scipy coverage matplotlib lxml +RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matplotlib lxml -ARG BASIC_PIP_PKGS="numpy pyarrow>=15.0.0 six==1.16.0 pandas==2.2.2 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" +ARG BASIC_PIP_PKGS="numpy pyarrow>=15.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2 twine==3.4.1" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.28.3 googleapis-common-protos==1.65.0" From 23f276f64d6d18a2d7a72149474c07e96a78b6ec Mon Sep 17 00:00:00 2001 From: Mihailo Aleksic Date: Wed, 20 Nov 2024 10:45:15 +0100 Subject: [PATCH 25/58] [SPARK-50353][SQL] Refactor ResolveSQLOnFile ### What changes were proposed in this pull request? Refactoring of `ResolveSQLOnFile` rule. ### Why are the changes needed? Refactoring is needed for the Single-pass Analyzer project (please check [link](https://issues.apache.org/jira/browse/SPARK-49834)) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48894 from mihailoale-db/mihailoale-db/refactorresolvesqlonfile. Authored-by: Mihailo Aleksic Signed-off-by: Max Gekk --- .../sql/execution/datasources/rules.scala | 40 ++++++++++++------- 1 file changed, 25 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index eaad99ab05f8e..fcc3a257cd2dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -46,6 +46,29 @@ import org.apache.spark.util.ArrayImplicits._ * Replaces [[UnresolvedRelation]]s if the plan is for direct query on files. */ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { + object UnresolvedRelationResolution { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = { + plan match { + case u: UnresolvedRelation if maybeSQLFile(u) => + try { + val ds = resolveDataSource(u) + Some(LogicalRelation(ds.resolveRelation())) + } catch { + case _: ClassNotFoundException => None + case e: Exception if !e.isInstanceOf[AnalysisException] => + // the provider is valid, but failed to create a logical plan + u.failAnalysis( + errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", + messageParameters = Map("dataSourceType" -> u.multipartIdentifier.head), + cause = e + ) + } + case _ => + None + } + } + } + private def maybeSQLFile(u: UnresolvedRelation): Boolean = { conf.runSQLonFile && u.multipartIdentifier.size == 2 } @@ -87,21 +110,8 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { } catch { case _: ClassNotFoundException => r } - - case u: UnresolvedRelation if maybeSQLFile(u) => - try { - val ds = resolveDataSource(u) - LogicalRelation(ds.resolveRelation()) - } catch { - case _: ClassNotFoundException => u - case e: Exception if !e.isInstanceOf[AnalysisException] => - // the provider is valid, but failed to create a logical plan - u.failAnalysis( - errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", - messageParameters = Map("dataSourceType" -> u.multipartIdentifier.head), - cause = e - ) - } + case UnresolvedRelationResolution(resolvedRelation) => + resolvedRelation } } From 533b8cab827b944c1b04ea3ac61168821e5bf2cf Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Wed, 20 Nov 2024 20:15:46 +0900 Subject: [PATCH 26/58] [SPARK-50363][PYTHON][DOCS] Refine the docstring for datetime functions - part 1 ### What changes were proposed in this pull request? Refine the docstring for datetime functions - part 1 ### Why are the changes needed? improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only change ### How was this patch tested? updated doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48904 from zhengruifeng/py_doc_6. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 494 +++++++++++++++++------- 1 file changed, 361 insertions(+), 133 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 23bec6024c94e..f5ac8b2c45599 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -10137,7 +10137,7 @@ def extract(field: Column, source: "ColumnOrName") -> Column: ---------- field : :class:`~pyspark.sql.Column` selects which part of the source should be extracted. - source : :class:`~pyspark.sql.Column` or str + source : :class:`~pyspark.sql.Column` or column name a date/timestamp or interval column from where `field` should be extracted. Returns @@ -10145,19 +10145,30 @@ def extract(field: Column, source: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` a part of the date/timestamp or interval source. + See Also + -------- + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.date_part` + Examples -------- >>> import datetime + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(datetime.datetime(2015, 4, 8, 13, 8, 15),)], ['ts']) >>> df.select( - ... extract(lit('YEAR'), 'ts').alias('year'), - ... extract(lit('month'), 'ts').alias('month'), - ... extract(lit('WEEK'), 'ts').alias('week'), - ... extract(lit('D'), 'ts').alias('day'), - ... extract(lit('M'), 'ts').alias('minute'), - ... extract(lit('S'), 'ts').alias('second') - ... ).collect() - [Row(year=2015, month=4, week=15, day=8, minute=8, second=Decimal('15.000000'))] + ... '*', + ... sf.extract(sf.lit('YEAR'), 'ts').alias('year'), + ... sf.extract(sf.lit('month'), 'ts').alias('month'), + ... sf.extract(sf.lit('WEEK'), 'ts').alias('week'), + ... sf.extract(sf.lit('D'), df.ts).alias('day'), + ... sf.extract(sf.lit('M'), df.ts).alias('minute'), + ... sf.extract(sf.lit('S'), df.ts).alias('second') + ... ).show() + +-------------------+----+-----+----+---+------+---------+ + | ts|year|month|week|day|minute| second| + +-------------------+----+-----+----+---+------+---------+ + |2015-04-08 13:08:15|2015| 4| 15| 8| 8|15.000000| + +-------------------+----+-----+----+---+------+---------+ """ return _invoke_function_over_columns("extract", field, source) @@ -10174,7 +10185,7 @@ def date_part(field: Column, source: "ColumnOrName") -> Column: field : :class:`~pyspark.sql.Column` selects which part of the source should be extracted, and supported string values are as same as the fields of the equivalent function `extract`. - source : :class:`~pyspark.sql.Column` or str + source : :class:`~pyspark.sql.Column` or column name a date/timestamp or interval column from where `field` should be extracted. Returns @@ -10182,19 +10193,30 @@ def date_part(field: Column, source: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` a part of the date/timestamp or interval source. + See Also + -------- + :meth:`pyspark.sql.functions.datepart` + :meth:`pyspark.sql.functions.extract` + Examples -------- >>> import datetime + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(datetime.datetime(2015, 4, 8, 13, 8, 15),)], ['ts']) >>> df.select( - ... date_part(lit('YEAR'), 'ts').alias('year'), - ... date_part(lit('month'), 'ts').alias('month'), - ... date_part(lit('WEEK'), 'ts').alias('week'), - ... date_part(lit('D'), 'ts').alias('day'), - ... date_part(lit('M'), 'ts').alias('minute'), - ... date_part(lit('S'), 'ts').alias('second') - ... ).collect() - [Row(year=2015, month=4, week=15, day=8, minute=8, second=Decimal('15.000000'))] + ... '*', + ... sf.date_part(sf.lit('YEAR'), 'ts').alias('year'), + ... sf.date_part(sf.lit('month'), 'ts').alias('month'), + ... sf.date_part(sf.lit('WEEK'), 'ts').alias('week'), + ... sf.date_part(sf.lit('D'), df.ts).alias('day'), + ... sf.date_part(sf.lit('M'), df.ts).alias('minute'), + ... sf.date_part(sf.lit('S'), df.ts).alias('second') + ... ).show() + +-------------------+----+-----+----+---+------+---------+ + | ts|year|month|week|day|minute| second| + +-------------------+----+-----+----+---+------+---------+ + |2015-04-08 13:08:15|2015| 4| 15| 8| 8|15.000000| + +-------------------+----+-----+----+---+------+---------+ """ return _invoke_function_over_columns("date_part", field, source) @@ -10211,7 +10233,7 @@ def datepart(field: Column, source: "ColumnOrName") -> Column: field : :class:`~pyspark.sql.Column` selects which part of the source should be extracted, and supported string values are as same as the fields of the equivalent function `extract`. - source : :class:`~pyspark.sql.Column` or str + source : :class:`~pyspark.sql.Column` or column name a date/timestamp or interval column from where `field` should be extracted. Returns @@ -10219,19 +10241,30 @@ def datepart(field: Column, source: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` a part of the date/timestamp or interval source. + See Also + -------- + :meth:`pyspark.sql.functions.date_part` + :meth:`pyspark.sql.functions.extract` + Examples -------- >>> import datetime + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(datetime.datetime(2015, 4, 8, 13, 8, 15),)], ['ts']) >>> df.select( - ... datepart(lit('YEAR'), 'ts').alias('year'), - ... datepart(lit('month'), 'ts').alias('month'), - ... datepart(lit('WEEK'), 'ts').alias('week'), - ... datepart(lit('D'), 'ts').alias('day'), - ... datepart(lit('M'), 'ts').alias('minute'), - ... datepart(lit('S'), 'ts').alias('second') - ... ).collect() - [Row(year=2015, month=4, week=15, day=8, minute=8, second=Decimal('15.000000'))] + ... '*', + ... sf.datepart(sf.lit('YEAR'), 'ts').alias('year'), + ... sf.datepart(sf.lit('month'), 'ts').alias('month'), + ... sf.datepart(sf.lit('WEEK'), 'ts').alias('week'), + ... sf.datepart(sf.lit('D'), df.ts).alias('day'), + ... sf.datepart(sf.lit('M'), df.ts).alias('minute'), + ... sf.datepart(sf.lit('S'), df.ts).alias('second') + ... ).show() + +-------------------+----+-----+----+---+------+---------+ + | ts|year|month|week|day|minute| second| + +-------------------+----+-----+----+---+------+---------+ + |2015-04-08 13:08:15|2015| 4| 15| 8| 8|15.000000| + +-------------------+----+-----+----+---+------+---------+ """ return _invoke_function_over_columns("datepart", field, source) @@ -10248,11 +10281,11 @@ def make_date(year: "ColumnOrName", month: "ColumnOrName", day: "ColumnOrName") Parameters ---------- - year : :class:`~pyspark.sql.Column` or str + year : :class:`~pyspark.sql.Column` or column name The year to build the date - month : :class:`~pyspark.sql.Column` or str + month : :class:`~pyspark.sql.Column` or column name The month to build the date - day : :class:`~pyspark.sql.Column` or str + day : :class:`~pyspark.sql.Column` or column name The day to build the date Returns @@ -10260,11 +10293,22 @@ def make_date(year: "ColumnOrName", month: "ColumnOrName", day: "ColumnOrName") :class:`~pyspark.sql.Column` a date built from given parts. + See Also + -------- + :meth:`pyspark.sql.functions.make_timestamp` + :meth:`pyspark.sql.functions.make_timestamp_ltz` + :meth:`pyspark.sql.functions.make_timestamp_ntz` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(2020, 6, 26)], ['Y', 'M', 'D']) - >>> df.select(make_date(df.Y, df.M, df.D).alias("datefield")).collect() - [Row(datefield=datetime.date(2020, 6, 26))] + >>> df.select('*', sf.make_date(df.Y, 'M', df.D)).show() + +----+---+---+------------------+ + | Y| M| D|make_date(Y, M, D)| + +----+---+---+------------------+ + |2020| 6| 26| 2020-06-26| + +----+---+---+------------------+ """ return _invoke_function_over_columns("make_date", year, month, day) @@ -10282,9 +10326,9 @@ def date_add(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: Parameters ---------- - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name date column to work on. - days : :class:`~pyspark.sql.Column` or str or int + days : :class:`~pyspark.sql.Column` or column name or int how many days after the given date to calculate. Accepts negative value as well to calculate backwards in time. @@ -10293,15 +10337,37 @@ def date_add(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: :class:`~pyspark.sql.Column` a date after/before given number of days. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_sub` + :meth:`pyspark.sql.functions.datediff` + :meth:`pyspark.sql.functions.date_diff` + Examples -------- - >>> df = spark.createDataFrame([('2015-04-08', 2,)], ['dt', 'add']) - >>> df.select(date_add(df.dt, 1).alias('next_date')).collect() - [Row(next_date=datetime.date(2015, 4, 9))] - >>> df.select(date_add(df.dt, df.add.cast('integer')).alias('next_date')).collect() - [Row(next_date=datetime.date(2015, 4, 10))] - >>> df.select(date_add('dt', -1).alias('prev_date')).collect() - [Row(prev_date=datetime.date(2015, 4, 7))] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('2015-04-08', 2,)], 'struct') + >>> df.select('*', sf.date_add(df.dt, 1)).show() + +----------+---+---------------+ + | dt| a|date_add(dt, 1)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-09| + +----------+---+---------------+ + + >>> df.select('*', sf.date_add('dt', 'a')).show() + +----------+---+---------------+ + | dt| a|date_add(dt, a)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-10| + +----------+---+---------------+ + + >>> df.select('*', sf.date_add('dt', sf.lit(-1))).show() + +----------+---+----------------+ + | dt| a|date_add(dt, -1)| + +----------+---+----------------+ + |2015-04-08| 2| 2015-04-07| + +----------+---+----------------+ """ days = _enum_to_value(days) days = lit(days) if isinstance(days, int) else days @@ -10318,9 +10384,9 @@ def dateadd(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: Parameters ---------- - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name date column to work on. - days : :class:`~pyspark.sql.Column` or str or int + days : :class:`~pyspark.sql.Column` or column name or int how many days after the given date to calculate. Accepts negative value as well to calculate backwards in time. @@ -10329,37 +10395,38 @@ def dateadd(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: :class:`~pyspark.sql.Column` a date after/before given number of days. + See Also + -------- + :meth:`pyspark.sql.functions.date_add` + :meth:`pyspark.sql.functions.date_sub` + :meth:`pyspark.sql.functions.datediff` + :meth:`pyspark.sql.functions.date_diff` + :meth:`pyspark.sql.functions.timestamp_add` + Examples -------- >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('2015-04-08', 2,)], ['dt', 'add'] - ... ).select(sf.dateadd("dt", 1)).show() - +---------------+ - |date_add(dt, 1)| - +---------------+ - | 2015-04-09| - +---------------+ + >>> df = spark.createDataFrame([('2015-04-08', 2,)], 'struct') + >>> df.select('*', sf.dateadd(df.dt, 1)).show() + +----------+---+---------------+ + | dt| a|date_add(dt, 1)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-09| + +----------+---+---------------+ - >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('2015-04-08', 2,)], ['dt', 'add'] - ... ).select(sf.dateadd("dt", sf.lit(2))).show() - +---------------+ - |date_add(dt, 2)| - +---------------+ - | 2015-04-10| - +---------------+ + >>> df.select('*', sf.dateadd('dt', 'a')).show() + +----------+---+---------------+ + | dt| a|date_add(dt, a)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-10| + +----------+---+---------------+ - >>> import pyspark.sql.functions as sf - >>> spark.createDataFrame( - ... [('2015-04-08', 2,)], ['dt', 'add'] - ... ).select(sf.dateadd("dt", -1)).show() - +----------------+ - |date_add(dt, -1)| - +----------------+ - | 2015-04-07| - +----------------+ + >>> df.select('*', sf.dateadd('dt', sf.lit(-1))).show() + +----------+---+----------------+ + | dt| a|date_add(dt, -1)| + +----------+---+----------------+ + |2015-04-08| 2| 2015-04-07| + +----------+---+----------------+ """ days = _enum_to_value(days) days = lit(days) if isinstance(days, int) else days @@ -10379,9 +10446,9 @@ def date_sub(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: Parameters ---------- - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name date column to work on. - days : :class:`~pyspark.sql.Column` or str or int + days : :class:`~pyspark.sql.Column` or column name or int how many days before the given date to calculate. Accepts negative value as well to calculate forward in time. @@ -10390,15 +10457,37 @@ def date_sub(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: :class:`~pyspark.sql.Column` a date before/after given number of days. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_add` + :meth:`pyspark.sql.functions.datediff` + :meth:`pyspark.sql.functions.date_diff` + Examples -------- - >>> df = spark.createDataFrame([('2015-04-08', 2,)], ['dt', 'sub']) - >>> df.select(date_sub(df.dt, 1).alias('prev_date')).collect() - [Row(prev_date=datetime.date(2015, 4, 7))] - >>> df.select(date_sub(df.dt, df.sub.cast('integer')).alias('prev_date')).collect() - [Row(prev_date=datetime.date(2015, 4, 6))] - >>> df.select(date_sub('dt', -1).alias('next_date')).collect() - [Row(next_date=datetime.date(2015, 4, 9))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-04-08', 2,)], 'struct') + >>> df.select('*', sf.date_sub(df.dt, 1)).show() + +----------+---+---------------+ + | dt| a|date_sub(dt, 1)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-07| + +----------+---+---------------+ + + >>> df.select('*', sf.date_sub('dt', 'a')).show() + +----------+---+---------------+ + | dt| a|date_sub(dt, a)| + +----------+---+---------------+ + |2015-04-08| 2| 2015-04-06| + +----------+---+---------------+ + + >>> df.select('*', sf.date_sub('dt', sf.lit(-1))).show() + +----------+---+----------------+ + | dt| a|date_sub(dt, -1)| + +----------+---+----------------+ + |2015-04-08| 2| 2015-04-09| + +----------+---+----------------+ """ days = _enum_to_value(days) days = lit(days) if isinstance(days, int) else days @@ -10417,9 +10506,9 @@ def datediff(end: "ColumnOrName", start: "ColumnOrName") -> Column: Parameters ---------- - end : :class:`~pyspark.sql.Column` or str + end : :class:`~pyspark.sql.Column` or column name to date column to work on. - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name from date column to work on. Returns @@ -10427,11 +10516,31 @@ def datediff(end: "ColumnOrName", start: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` difference in days between two dates. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_add` + :meth:`pyspark.sql.functions.date_sub` + :meth:`pyspark.sql.functions.date_diff` + :meth:`pyspark.sql.functions.timestamp_diff` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) - >>> df.select(datediff(df.d2, df.d1).alias('diff')).collect() - [Row(diff=32)] + >>> df.select('*', sf.datediff('d1', 'd2')).show() + +----------+----------+----------------+ + | d1| d2|datediff(d1, d2)| + +----------+----------+----------------+ + |2015-04-08|2015-05-10| -32| + +----------+----------+----------------+ + + >>> df.select('*', sf.datediff(df.d2, df.d1)).show() + +----------+----------+----------------+ + | d1| d2|datediff(d2, d1)| + +----------+----------+----------------+ + |2015-04-08|2015-05-10| 32| + +----------+----------+----------------+ """ return _invoke_function_over_columns("datediff", end, start) @@ -10445,9 +10554,9 @@ def date_diff(end: "ColumnOrName", start: "ColumnOrName") -> Column: Parameters ---------- - end : :class:`~pyspark.sql.Column` or str + end : :class:`~pyspark.sql.Column` or column name to date column to work on. - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name from date column to work on. Returns @@ -10455,11 +10564,30 @@ def date_diff(end: "ColumnOrName", start: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` difference in days between two dates. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_add` + :meth:`pyspark.sql.functions.date_sub` + :meth:`pyspark.sql.functions.datediff` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2']) - >>> df.select(date_diff(df.d2, df.d1).alias('diff')).collect() - [Row(diff=32)] + >>> df.select('*', sf.date_diff('d1', 'd2')).show() + +----------+----------+-----------------+ + | d1| d2|date_diff(d1, d2)| + +----------+----------+-----------------+ + |2015-04-08|2015-05-10| -32| + +----------+----------+-----------------+ + + >>> df.select('*', sf.date_diff(df.d2, df.d1)).show() + +----------+----------+-----------------+ + | d1| d2|date_diff(d2, d1)| + +----------+----------+-----------------+ + |2015-04-08|2015-05-10| 32| + +----------+----------+-----------------+ """ return _invoke_function_over_columns("date_diff", end, start) @@ -10473,7 +10601,7 @@ def date_from_unix_date(days: "ColumnOrName") -> Column: Parameters ---------- - days : :class:`~pyspark.sql.Column` or str + days : :class:`~pyspark.sql.Column` or column name the target column to work on. Returns @@ -10481,15 +10609,22 @@ def date_from_unix_date(days: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` the date from the number of days since 1970-01-01. + See Also + -------- + :meth:`pyspark.sql.functions.from_unixtime` + Examples -------- - >>> df = spark.range(1) - >>> df.select(date_from_unix_date(lit(1))).show() - +----------------------+ - |date_from_unix_date(1)| - +----------------------+ - | 1970-01-02| - +----------------------+ + >>> import pyspark.sql.functions as sf + >>> spark.range(4).select('*', sf.date_from_unix_date('id')).show() + +---+-----------------------+ + | id|date_from_unix_date(id)| + +---+-----------------------+ + | 0| 1970-01-01| + | 1| 1970-01-02| + | 2| 1970-01-03| + | 3| 1970-01-04| + +---+-----------------------+ """ return _invoke_function_over_columns("date_from_unix_date", days) @@ -10507,9 +10642,9 @@ def add_months(start: "ColumnOrName", months: Union["ColumnOrName", int]) -> Col Parameters ---------- - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name date column to work on. - months : :class:`~pyspark.sql.Column` or str or int + months : :class:`~pyspark.sql.Column` or column name or int how many months after the given date to calculate. Accepts negative value as well to calculate backwards. @@ -10518,15 +10653,35 @@ def add_months(start: "ColumnOrName", months: Union["ColumnOrName", int]) -> Col :class:`~pyspark.sql.Column` a date after/before given number of months. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_add` + Examples -------- - >>> df = spark.createDataFrame([('2015-04-08', 2)], ['dt', 'add']) - >>> df.select(add_months(df.dt, 1).alias('next_month')).collect() - [Row(next_month=datetime.date(2015, 5, 8))] - >>> df.select(add_months(df.dt, df.add.cast('integer')).alias('next_month')).collect() - [Row(next_month=datetime.date(2015, 6, 8))] - >>> df.select(add_months('dt', -2).alias('prev_month')).collect() - [Row(prev_month=datetime.date(2015, 2, 8))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-04-08', 2,)], 'struct') + >>> df.select('*', sf.add_months(df.dt, 1)).show() + +----------+---+-----------------+ + | dt| a|add_months(dt, 1)| + +----------+---+-----------------+ + |2015-04-08| 2| 2015-05-08| + +----------+---+-----------------+ + + >>> df.select('*', sf.add_months('dt', 'a')).show() + +----------+---+-----------------+ + | dt| a|add_months(dt, a)| + +----------+---+-----------------+ + |2015-04-08| 2| 2015-06-08| + +----------+---+-----------------+ + + >>> df.select('*', sf.add_months('dt', sf.lit(-1))).show() + +----------+---+------------------+ + | dt| a|add_months(dt, -1)| + +----------+---+------------------+ + |2015-04-08| 2| 2015-03-08| + +----------+---+------------------+ """ months = _enum_to_value(months) months = lit(months) if isinstance(months, int) else months @@ -10549,9 +10704,9 @@ def months_between(date1: "ColumnOrName", date2: "ColumnOrName", roundOff: bool Parameters ---------- - date1 : :class:`~pyspark.sql.Column` or str + date1 : :class:`~pyspark.sql.Column` or column name first date column. - date2 : :class:`~pyspark.sql.Column` or str + date2 : :class:`~pyspark.sql.Column` or column name second date column. roundOff : bool, optional whether to round (to 8 digits) the final value or not (default: True). @@ -10563,11 +10718,28 @@ def months_between(date1: "ColumnOrName", date2: "ColumnOrName", roundOff: bool Examples -------- - >>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['date1', 'date2']) - >>> df.select(months_between(df.date1, df.date2).alias('months')).collect() - [Row(months=3.94959677)] - >>> df.select(months_between(df.date1, df.date2, False).alias('months')).collect() - [Row(months=3.9495967741935485)] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['d1', 'd2']) + >>> df.select('*', sf.months_between(df.d1, df.d2)).show() + +-------------------+----------+----------------------------+ + | d1| d2|months_between(d1, d2, true)| + +-------------------+----------+----------------------------+ + |1997-02-28 10:30:00|1996-10-30| 3.94959677| + +-------------------+----------+----------------------------+ + + >>> df.select('*', sf.months_between('d2', 'd1')).show() + +-------------------+----------+----------------------------+ + | d1| d2|months_between(d2, d1, true)| + +-------------------+----------+----------------------------+ + |1997-02-28 10:30:00|1996-10-30| -3.94959677| + +-------------------+----------+----------------------------+ + + >>> df.select('*', sf.months_between('d1', df.d2, False)).show() + +-------------------+----------+-----------------------------+ + | d1| d2|months_between(d1, d2, false)| + +-------------------+----------+-----------------------------+ + |1997-02-28 10:30:00|1996-10-30| 3.9495967741935...| + +-------------------+----------+-----------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -10997,9 +11169,9 @@ def trunc(date: "ColumnOrName", format: str) -> Column: Parameters ---------- - date : :class:`~pyspark.sql.Column` or str + date : :class:`~pyspark.sql.Column` or column name input column of values to truncate. - format : str + format : literal string 'year', 'yyyy', 'yy' to truncate by year, or 'month', 'mon', 'mm' to truncate by month Other options are: 'week', 'quarter' @@ -11009,13 +11181,27 @@ def trunc(date: "ColumnOrName", format: str) -> Column: :class:`~pyspark.sql.Column` truncated date. + See Also + -------- + :meth:`pyspark.sql.functions.date_trunc` + Examples -------- - >>> df = spark.createDataFrame([('1997-02-28',)], ['d']) - >>> df.select(trunc(df.d, 'year').alias('year')).collect() - [Row(year=datetime.date(1997, 1, 1))] - >>> df.select(trunc(df.d, 'mon').alias('month')).collect() - [Row(month=datetime.date(1997, 2, 1))] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('1997-02-28',)], ['dt']) + >>> df.select('*', sf.trunc(df.dt, 'year')).show() + +----------+---------------+ + | dt|trunc(dt, year)| + +----------+---------------+ + |1997-02-28| 1997-01-01| + +----------+---------------+ + + >>> df.select('*', sf.trunc('dt', 'mon')).show() + +----------+--------------+ + | dt|trunc(dt, mon)| + +----------+--------------+ + |1997-02-28| 1997-02-01| + +----------+--------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -11034,13 +11220,13 @@ def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: Parameters ---------- - format : str + format : literal string 'year', 'yyyy', 'yy' to truncate by year, 'month', 'mon', 'mm' to truncate by month, 'day', 'dd' to truncate by day, Other options are: 'microsecond', 'millisecond', 'second', 'minute', 'hour', 'week', 'quarter' - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name input column of values to truncate. Returns @@ -11048,13 +11234,27 @@ def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` truncated timestamp. + See Also + -------- + :meth:`pyspark.sql.functions.trunc` + Examples -------- - >>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['t']) - >>> df.select(date_trunc('year', df.t).alias('year')).collect() - [Row(year=datetime.datetime(1997, 1, 1, 0, 0))] - >>> df.select(date_trunc('mon', df.t).alias('month')).collect() - [Row(month=datetime.datetime(1997, 2, 1, 0, 0))] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['ts']) + >>> df.select('*', sf.date_trunc('year', df.ts)).show() + +-------------------+--------------------+ + | ts|date_trunc(year, ts)| + +-------------------+--------------------+ + |1997-02-28 05:02:11| 1997-01-01 00:00:00| + +-------------------+--------------------+ + + >>> df.select('*', sf.date_trunc('mon', 'ts')).show() + +-------------------+-------------------+ + | ts|date_trunc(mon, ts)| + +-------------------+-------------------+ + |1997-02-28 05:02:11|1997-02-01 00:00:00| + +-------------------+-------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -11074,9 +11274,9 @@ def next_day(date: "ColumnOrName", dayOfWeek: str) -> Column: Parameters ---------- - date : :class:`~pyspark.sql.Column` or str + date : :class:`~pyspark.sql.Column` or column name target column to compute on. - dayOfWeek : str + dayOfWeek : literal string day of the week, case-insensitive, accepts: "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun" @@ -11087,9 +11287,21 @@ def next_day(date: "ColumnOrName", dayOfWeek: str) -> Column: Examples -------- - >>> df = spark.createDataFrame([('2015-07-27',)], ['d']) - >>> df.select(next_day(df.d, 'Sun').alias('date')).collect() - [Row(date=datetime.date(2015, 8, 2))] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('2015-07-27',)], ['dt']) + >>> df.select('*', sf.next_day(df.dt, 'Sun')).show() + +----------+-----------------+ + | dt|next_day(dt, Sun)| + +----------+-----------------+ + |2015-07-27| 2015-08-02| + +----------+-----------------+ + + >>> df.select('*', sf.next_day('dt', 'Sat')).show() + +----------+-----------------+ + | dt|next_day(dt, Sat)| + +----------+-----------------+ + |2015-07-27| 2015-08-01| + +----------+-----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -11108,7 +11320,7 @@ def last_day(date: "ColumnOrName") -> Column: Parameters ---------- - date : :class:`~pyspark.sql.Column` or str + date : :class:`~pyspark.sql.Column` or column name target column to compute on. Returns @@ -11118,9 +11330,21 @@ def last_day(date: "ColumnOrName") -> Column: Examples -------- - >>> df = spark.createDataFrame([('1997-02-10',)], ['d']) - >>> df.select(last_day(df.d).alias('date')).collect() - [Row(date=datetime.date(1997, 2, 28))] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([('1997-02-10',)], ['dt']) + >>> df.select('*', sf.last_day(df.dt)).show() + +----------+------------+ + | dt|last_day(dt)| + +----------+------------+ + |1997-02-10| 1997-02-28| + +----------+------------+ + + >>> df.select('*', sf.last_day('dt')).show() + +----------+------------+ + | dt|last_day(dt)| + +----------+------------+ + |1997-02-10| 1997-02-28| + +----------+------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -11151,6 +11375,10 @@ def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss" :class:`~pyspark.sql.Column` formatted timestamp as string. + See Also + -------- + :meth:`pyspark.sql.functions.date_from_unix_date` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") From 81a56df543f6f00476ff0a0cba84a9a54402f2b2 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 20 Nov 2024 20:22:32 +0800 Subject: [PATCH 27/58] [SPARK-50362][PYTHON][ML] Skip `CrossValidatorTests` if `torch/torcheval` not installed ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/48900 skipped relevant test cases on macOS by no longer installing dependencies related to `torch` and `deepseed`. Subsequently, some test cases in `ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests` failed due to the absence of `torch/torcheval`: ``` ERROR (3.456s) test_crossvalidator_with_fold_col (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_crossvalidator_with_fold_col) ... ERROR (2.550s) test_fit_maximize_metric (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_fit_maximize_metric) ... ERROR (0.550s) test_fit_minimize_metric (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_fit_minimize_metric) ... ERROR (0.544s) test_gen_avg_and_std_metrics (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_gen_avg_and_std_metrics) ... ok (0.539s) ====================================================================== ERROR [9.991s]: test_copy (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_copy) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/runner/work/spark/spark/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py", line 119, in test_copy cvModel = cv.fit(dataset) ^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/base.py", line 105, in fit return self._fit(dataset) ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 435, in _fit for j, metric in pool.imap_unordered(lambda f: f(), tasks): File "/opt/homebrew/Cellar/python3.11/3.11.10/Frameworks/Python.framework/Versions/3.11/lib/python3.11/multiprocessing/pool.py", line 873, in next raise value File "/opt/homebrew/Cellar/python3.11/3.11.10/Frameworks/Python.framework/Versions/3.11/lib/python3.11/multiprocessing/pool.py", line 125, in worker result = (True, func(*args, **kwds)) ^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 435, in for j, metric in pool.imap_unordered(lambda f: f(), tasks): ^^^ File "/Users/runner/work/spark/spark/python/pyspark/util.py", line 423, in wrapped return f(*args, **kwargs) # type: ignore[misc, operator] ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 186, in single_task metric = evaluator.evaluate( ^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/base.py", line 254, in evaluate return self._evaluate(dataset) ^^^^^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/evaluation.py", line 57, in _evaluate torch_metric = self._get_torch_metric() ^^^^^^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/evaluation.py", line 125, in _get_torch_metric import torcheval.metrics as torchmetrics ModuleNotFoundError: No module named 'torcheval' ====================================================================== ERROR [3.456s]: test_crossvalidator_on_pipeline (pyspark.ml.tests.connect.test_legacy_mode_tuning.CrossValidatorTests.test_crossvalidator_on_pipeline) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/runner/work/spark/spark/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py", line 210, in test_crossvalidator_on_pipeline cv_model = cv.fit(train_dataset) ^^^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/base.py", line 105, in fit return self._fit(dataset) ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 435, in _fit for j, metric in pool.imap_unordered(lambda f: f(), tasks): File "/opt/homebrew/Cellar/python3.11/3.11.10/Frameworks/Python.framework/Versions/3.11/lib/python3.11/multiprocessing/pool.py", line 873, in next raise value File "/opt/homebrew/Cellar/python3.11/3.11.10/Frameworks/Python.framework/Versions/3.11/lib/python3.11/multiprocessing/pool.py", line 125, in worker result = (True, func(*args, **kwds)) ^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 435, in for j, metric in pool.imap_unordered(lambda f: f(), tasks): ^^^ File "/Users/runner/work/spark/spark/python/pyspark/util.py", line 423, in wrapped return f(*args, **kwargs) # type: ignore[misc, operator] ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/tuning.py", line 185, in single_task model = estimator.fit(train, param_map) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/base.py", line 103, in fit return self.copy(params)._fit(dataset) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/pipeline.py", line 201, in _fit model = stage.fit(dataset) # type: ignore[attr-defined] ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/base.py", line 105, in fit return self._fit(dataset) ^^^^^^^^^^^^^^^^^^ File "/Users/runner/work/spark/spark/python/pyspark/ml/connect/classification.py", line 219, in _fit import torch ModuleNotFoundError: No module named 'torch' .... ``` Therefore, this pull request adds corresponding conditions to `CrossValidatorTests` to skip tests when `torch/torcheval` is not installed. ### Why are the changes neede Skip `CrossValidatorTests` if `torch/torcheval` not installed ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #48901 from LuciferYang/pyspark-on-macos-split-mltest. Authored-by: yangjie01 Signed-off-by: Ruifeng Zheng --- .../tests/connect/test_legacy_mode_tuning.py | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py b/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py index 0e3482157be50..33cc39d3319db 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py @@ -37,6 +37,22 @@ have_sklearn = False sklearn_requirement_message = "No sklearn found" +have_torch = True +torch_requirement_message = None +try: + import torch # noqa: F401 +except ImportError: + have_torch = False + torch_requirement_message = "torch is required" + +have_torcheval = True +torcheval_requirement_message = None +try: + import torcheval # noqa: F401 +except ImportError: + have_torcheval = False + torcheval_requirement_message = "torcheval is required" + if should_test_connect: import pandas as pd @@ -280,9 +296,15 @@ def test_crossvalidator_with_fold_col(self): @unittest.skipIf( - not should_test_connect or not have_sklearn or is_remote_only(), + not should_test_connect + or not have_sklearn + or not have_torch + or not have_torcheval + or is_remote_only(), connect_requirement_message or sklearn_requirement_message + or torch_requirement_message + or torcheval_requirement_message or "pyspark-connect cannot test classic Spark", ) class CrossValidatorTests(CrossValidatorTestsMixin, unittest.TestCase): From 6ee53da5f356232e2026a67c8408de38c625038e Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 20 Nov 2024 21:03:14 +0800 Subject: [PATCH 28/58] [SPARK-50258][SQL] Fix output column order changed issue after AQE optimization ### What changes were proposed in this pull request? The root cause of this issue is the planner turns `Limit` + `Sort` into `TakeOrderedAndProjectExec` which adds an additional `Project` that does not exist in the logical plan. We shouldn't use this additional `Project` to optimize out other `Project`s, otherwise when AQE turns physical plan back to logical plan, we lose the `Project` and may mess up the output column order. This PR makes it does not remove redundant projects if AEQ is enabled and projectList is the same as child output in `TakeOrderedAndProjectExec`. ### Why are the changes needed? Fix potential data issue and avoid Spark Driver crash: ``` # more hs_err_pid193136.log # # A fatal error has been detected by the Java Runtime Environment: # # SIGSEGV (0xb) at pc=0x00007f9d14841bc0, pid=193136, tid=223205 # # JRE version: OpenJDK Runtime Environment Zulu17.36+18-SA (17.0.4.1+1) (build 17.0.4.1+1-LTS) # Java VM: OpenJDK 64-Bit Server VM Zulu17.36+18-SA (17.0.4.1+1-LTS, mixed mode, sharing, tiered, compressed class ptrs, g1 gc, linux-amd64) # Problematic frame: # v ~StubRoutines::jint_disjoint_arraycopy_avx3 # # Core dump will be written. Default location: /apache/spark-release/3.5.0-20241105/spark/core.193136 ... ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48789 from wangyum/SPARK-50258. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan --- .../execution/RemoveRedundantProjects.scala | 8 ++++++- .../adaptive/AdaptiveQueryExecSuite.scala | 23 ++++++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala index 8f4ce0f49a89a..69230fd7b3343 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/RemoveRedundantProjects.scala @@ -58,7 +58,13 @@ object RemoveRedundantProjects extends Rule[SparkPlan] { p.mapChildren(removeProject(_, false)) } case op: TakeOrderedAndProjectExec => - op.mapChildren(removeProject(_, false)) + // The planner turns Limit + Sort into TakeOrderedAndProjectExec which adds an additional + // Project that does not exist in the logical plan. We shouldn't use this additional Project + // to optimize out other Projects, otherwise when AQE turns physical plan back to + // logical plan, we lose the Project and may mess up the output column order. So column + // ordering is required if AQE is enabled and projectList is the same as child output. + val requireColOrdering = conf.adaptiveExecutionEnabled && op.projectList == op.child.output + op.mapChildren(removeProject(_, requireColOrdering)) case a: BaseAggregateExec => // BaseAggregateExec require specific column ordering when mode is Final or PartialMerge. // See comments in BaseAggregateExec inputAttributes method. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 1df045764d8b9..ad28fd5176d99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} -import org.apache.spark.sql.execution.{CollectLimitExec, ColumnarToRowExec, EmptyRelationExec, PartialReducerPartitionSpec, QueryExecution, ReusedSubqueryExec, ShuffledRowRDD, SortExec, SparkPlan, SparkPlanInfo, UnaryExecNode, UnionExec} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.BaseAggregateExec import org.apache.spark.sql.execution.columnar.{InMemoryTableScanExec, InMemoryTableScanLike} import org.apache.spark.sql.execution.command.DataWritingCommandExec @@ -43,6 +43,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ENSURE_RE import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec, ShuffledJoin, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.execution.ui.{SparkListenerSQLAdaptiveExecutionUpdate, SparkListenerSQLAdaptiveSQLMetricUpdates, SparkListenerSQLExecutionStart} +import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode @@ -3086,6 +3087,26 @@ class AdaptiveQueryExecSuite } } } + + test("SPARK-50258: Fix output column order changed issue after AQE optimization") { + withTable("t") { + sql("SELECT course, year, earnings FROM courseSales").write.saveAsTable("t") + val df = sql( + """ + |SELECT year, course, earnings, SUM(earnings) OVER (ORDER BY year, course) AS balance + |FROM t ORDER BY year, course + |LIMIT 100 + |""".stripMargin) + df.collect() + + val plan = df.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec] + assert(plan.inputPlan.isInstanceOf[TakeOrderedAndProjectExec]) + assert(plan.finalPhysicalPlan.isInstanceOf[WindowExec]) + plan.inputPlan.output.zip(plan.finalPhysicalPlan.output).foreach { case (o1, o2) => + assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") + } + } + } } /** From 30d0b011ef877969d070f7ec9f60a633cc2ed319 Mon Sep 17 00:00:00 2001 From: Karim Ramadan Date: Wed, 20 Nov 2024 17:00:57 +0100 Subject: [PATCH 29/58] [SPARK-50364][SQL] Implement serialization for LocalDateTime type in Row.jsonValue ### What changes were proposed in this pull request? In this Pull request, I propose to add a LocalDateTime serializer to the Row.jsonValue method ```scala case (d: LocalDateTime, _) => JString(timestampFormatter.format(d)) ``` In order to enable JSON serialization of _TimestampNTZType_ columns ### Why are the changes needed? Currently trying to serialize a Row containing a _TimestampNTZType_ column results in an error: ```java [FAILED_ROW_TO_JSON] Failed to convert the row value '2018-05-14T12:13' of the class class java.time.LocalDateTime to the target SQL type "TIMESTAMPNTZTYPE" in the JSON format. SQLSTATE: 2203G org.apache.spark.SparkIllegalArgumentException: [FAILED_ROW_TO_JSON] Failed to convert the row value '2018-05-14T12:13' of the class class java.time.LocalDateTime to the target SQL type "TIMESTAMPNTZTYPE" in the JSON format. SQLSTATE: 2203G at org.apache.spark.sql.Row.toJson$1(Row.scala:663) at org.apache.spark.sql.Row.toJson$1(Row.scala:651) at org.apache.spark.sql.Row.jsonValue(Row.scala:665) at org.apache.spark.sql.Row.jsonValue$(Row.scala:598) at org.apache.spark.sql.catalyst.expressions.GenericRow.jsonValue(rows.scala:28) at org.apache.spark.sql.RowJsonSuite.$anonfun$testJson$1(RowJsonSuite.scala:41) ``` How to reproduce the issue: ```scala import org.apache.spark.sql.Row import java.time.LocalDateTime val r = Row.fromSeq(LocalDateTime.of(2024,8,10,12,33) :: Nil) r.json r.prettyJson ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tests were added to the already existing RowJsonSuite.scala class ### Was this patch authored or co-authored using generative AI tooling? No Closes #48905 from karim-ramadan/bugfix/LocalDateTime-support-in-Row-json. Authored-by: Karim Ramadan Signed-off-by: Max Gekk --- sql/api/src/main/scala/org/apache/spark/sql/Row.scala | 3 ++- .../src/test/scala/org/apache/spark/sql/RowJsonSuite.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala index aa14115453aea..764bdb17b37e2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} -import java.time.{Instant, LocalDate} +import java.time.{Instant, LocalDate, LocalDateTime} import java.util.Base64 import scala.collection.mutable @@ -627,6 +627,7 @@ trait Row extends Serializable { case (d: Date, _) => JString(dateFormatter.format(d)) case (i: Instant, _) => JString(timestampFormatter.format(i)) case (t: Timestamp, _) => JString(timestampFormatter.format(t)) + case (d: LocalDateTime, _) => JString(timestampFormatter.format(d)) case (i: CalendarInterval, _) => JString(i.toString) case (a: Array[_], ArrayType(elementType, _)) => iteratorToJsonArray(a.iterator, elementType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala index cf50063baa13c..49138532616e7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RowJsonSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} -import java.time.LocalDate +import java.time.{LocalDate, LocalDateTime} import org.json4s.JsonAST.{JArray, JBool, JDecimal, JDouble, JLong, JNull, JObject, JString, JValue} @@ -84,6 +84,7 @@ class RowJsonSuite extends SparkFunSuite { Timestamp.valueOf("2017-05-30 10:22:03.00").toInstant, TimestampType, JString("2017-05-30 10:22:03")) + testJson(LocalDateTime.of(2018, 5, 14, 12, 13), TimestampNTZType, JString("2018-05-14 12:13:00")) // Complex types testJson( From ad46db4ef671d8829dfffba2780ba0f6b4f4e43d Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Wed, 20 Nov 2024 13:22:51 -0800 Subject: [PATCH 30/58] [SPARK-50130][SQL][FOLLOWUP] Make Encoder generation lazy ### What changes were proposed in this pull request? Makes Encoder generation lazy. ### Why are the changes needed? The encoder with empty schema for lazy plan could cause unexpected behavior. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48829 from ueshin/issues/SPARK-50130/lazy_encoder. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../scala/org/apache/spark/sql/Dataset.scala | 35 +++++++------------ .../spark/sql/DataFrameSubquerySuite.scala | 15 +++++--- .../scala/org/apache/spark/sql/UDFSuite.scala | 2 +- 3 files changed, 25 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 500a4c7c4d9bc..4766a74308a1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -95,13 +95,8 @@ private[sql] object Dataset { def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = sparkSession.withActive { val qe = sparkSession.sessionState.executePlan(logicalPlan) - val encoder = if (qe.isLazyAnalysis) { - RowEncoder.encoderFor(new StructType()) - } else { - qe.assertAnalyzed() - RowEncoder.encoderFor(qe.analyzed.schema) - } - new Dataset[Row](qe, encoder) + if (!qe.isLazyAnalysis) qe.assertAnalyzed() + new Dataset[Row](qe, () => RowEncoder.encoderFor(qe.analyzed.schema)) } def ofRows( @@ -111,13 +106,8 @@ private[sql] object Dataset { sparkSession.withActive { val qe = new QueryExecution( sparkSession, logicalPlan, shuffleCleanupMode = shuffleCleanupMode) - val encoder = if (qe.isLazyAnalysis) { - RowEncoder.encoderFor(new StructType()) - } else { - qe.assertAnalyzed() - RowEncoder.encoderFor(qe.analyzed.schema) - } - new Dataset[Row](qe, encoder) + if (!qe.isLazyAnalysis) qe.assertAnalyzed() + new Dataset[Row](qe, () => RowEncoder.encoderFor(qe.analyzed.schema)) } /** A variant of ofRows that allows passing in a tracker so we can track query parsing time. */ @@ -129,13 +119,8 @@ private[sql] object Dataset { : DataFrame = sparkSession.withActive { val qe = new QueryExecution( sparkSession, logicalPlan, tracker, shuffleCleanupMode = shuffleCleanupMode) - val encoder = if (qe.isLazyAnalysis) { - RowEncoder.encoderFor(new StructType()) - } else { - qe.assertAnalyzed() - RowEncoder.encoderFor(qe.analyzed.schema) - } - new Dataset[Row](qe, encoder) + if (!qe.isLazyAnalysis) qe.assertAnalyzed() + new Dataset[Row](qe, () => RowEncoder.encoderFor(qe.analyzed.schema)) } } @@ -229,7 +214,7 @@ private[sql] object Dataset { @Stable class Dataset[T] private[sql]( @DeveloperApi @Unstable @transient val queryExecution: QueryExecution, - @DeveloperApi @Unstable @transient val encoder: Encoder[T]) + @transient encoderGenerator: () => Encoder[T]) extends api.Dataset[T] { type DS[U] = Dataset[U] @@ -252,6 +237,10 @@ class Dataset[T] private[sql]( // Note for Spark contributors: if adding or updating any action in `Dataset`, please make sure // you wrap it with `withNewExecutionId` if this actions doesn't call other action. + private[sql] def this(queryExecution: QueryExecution, encoder: Encoder[T]) = { + this(queryExecution, () => encoder) + } + def this(sparkSession: SparkSession, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { this(sparkSession.sessionState.executePlan(logicalPlan), encoder) } @@ -274,6 +263,8 @@ class Dataset[T] private[sql]( } } + @DeveloperApi @Unstable @transient lazy val encoder: Encoder[T] = encoderGenerator() + /** * Expose the encoder as implicit so it can be used to construct new Dataset objects that have * the same external type. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index 5a065d7e73b1c..d656c36ce842a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -54,11 +54,18 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } test("unanalyzable expression") { - val exception = intercept[AnalysisException] { - spark.range(1).select($"id" === $"id".outer()).schema - } + val sub = spark.range(1).select($"id" === $"id".outer()) + + checkError( + intercept[AnalysisException](sub.schema), + condition = "UNANALYZABLE_EXPRESSION", + parameters = Map("expr" -> "\"outer(id)\""), + queryContext = + Array(ExpectedContext(fragment = "outer", callSitePattern = getCurrentClassCallSitePattern)) + ) + checkError( - exception, + intercept[AnalysisException](sub.encoder), condition = "UNANALYZABLE_EXPRESSION", parameters = Map("expr" -> "\"outer(id)\""), queryContext = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index d550d0f94f236..18af2fcb0ee73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -1205,7 +1205,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { dt ) checkError( - intercept[AnalysisException](spark.range(1).select(f())), + intercept[AnalysisException](spark.range(1).select(f()).encoder), condition = "UNSUPPORTED_DATA_TYPE_FOR_ENCODER", sqlState = "0A000", parameters = Map("dataType" -> s"\"${dt.sql}\"") From a40919912f5ce7f63fff2907b30e473dd4155227 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 21 Nov 2024 14:57:55 +0900 Subject: [PATCH 31/58] [SPARK-50376][PYTHON][ML][TESTS] Centralize the dependency check in ML tests ### What changes were proposed in this pull request? Centralize the dependency check in ML tests ### Why are the changes needed? deduplicate code ### Does this PR introduce _any_ user-facing change? no, test-only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48911 from zhengruifeng/py_centralize_ml_dep. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- .../tests/test_deepspeed_distributor.py | 11 ++--- .../connect/test_connect_classification.py | 8 +-- .../tests/connect/test_connect_evaluation.py | 9 +--- .../ml/tests/connect/test_connect_feature.py | 9 +--- .../ml/tests/connect/test_connect_pipeline.py | 8 +-- .../ml/tests/connect/test_connect_tuning.py | 5 +- .../test_legacy_mode_classification.py | 11 ++--- .../connect/test_legacy_mode_evaluation.py | 9 +--- .../tests/connect/test_legacy_mode_feature.py | 5 +- .../connect/test_legacy_mode_pipeline.py | 5 +- .../tests/connect/test_legacy_mode_tuning.py | 37 +++++--------- .../connect/test_parity_torch_data_loader.py | 9 +--- .../connect/test_parity_torch_distributor.py | 9 +--- .../ml/torch/tests/test_data_loader.py | 13 ++--- .../ml/torch/tests/test_distributor.py | 18 +++---- python/pyspark/testing/utils.py | 49 ++++++++++++------- 16 files changed, 71 insertions(+), 144 deletions(-) diff --git a/python/pyspark/ml/deepspeed/tests/test_deepspeed_distributor.py b/python/pyspark/ml/deepspeed/tests/test_deepspeed_distributor.py index 590e541c3842a..66a9b553cc751 100644 --- a/python/pyspark/ml/deepspeed/tests/test_deepspeed_distributor.py +++ b/python/pyspark/ml/deepspeed/tests/test_deepspeed_distributor.py @@ -30,12 +30,7 @@ set_up_test_dirs, get_distributed_mode_conf, ) - -have_deepspeed = True -try: - import deepspeed # noqa: F401 -except ImportError: - have_deepspeed = False +from pyspark.testing.utils import have_deepspeed, deepspeed_requirement_message class DeepspeedTorchDistributorUnitTests(unittest.TestCase): @@ -219,7 +214,7 @@ def pythagorean_thm(x : int, y: int): # type: ignore # and inference, the hope is to switch out the training # and file for the tests with more realistic testing # that use Deepspeed constructs. -@unittest.skipIf(not have_deepspeed, "deepspeed is required for these tests") +@unittest.skipIf(not have_deepspeed, deepspeed_requirement_message) class DeepspeedTorchDistributorDistributedEndToEnd(unittest.TestCase): @classmethod def setUpClass(cls) -> None: @@ -259,7 +254,7 @@ def test_pytorch_file_e2e(self) -> None: dist.run(cp_path, 2, 5) -@unittest.skipIf(not have_deepspeed, "deepspeed is required for these tests") +@unittest.skipIf(not have_deepspeed, deepspeed_requirement_message) class DeepspeedDistributorLocalEndToEndTests(unittest.TestCase): @classmethod def setUpClass(cls) -> None: diff --git a/python/pyspark/ml/tests/connect/test_connect_classification.py b/python/pyspark/ml/tests/connect/test_connect_classification.py index 910d2d2ec42f9..d3e86a3fb9df7 100644 --- a/python/pyspark/ml/tests/connect/test_connect_classification.py +++ b/python/pyspark/ml/tests/connect/test_connect_classification.py @@ -23,13 +23,7 @@ from pyspark.sql import SparkSession from pyspark.ml.tests.connect.test_legacy_mode_classification import ClassificationTestsMixin from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -torch_requirement_message = "torch is required" -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False +from pyspark.testing.utils import have_torch, torch_requirement_message @unittest.skipIf( diff --git a/python/pyspark/ml/tests/connect/test_connect_evaluation.py b/python/pyspark/ml/tests/connect/test_connect_evaluation.py index 9acf5ae0ac44d..cabd8b5b50df1 100644 --- a/python/pyspark/ml/tests/connect/test_connect_evaluation.py +++ b/python/pyspark/ml/tests/connect/test_connect_evaluation.py @@ -20,19 +20,14 @@ from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -have_torcheval = True -try: - import torcheval # noqa: F401 -except ImportError: - have_torcheval = False +from pyspark.testing.utils import have_torcheval, torcheval_requirement_message if should_test_connect: from pyspark.ml.tests.connect.test_legacy_mode_evaluation import EvaluationTestsMixin @unittest.skipIf( not should_test_connect or not have_torcheval, - connect_requirement_message or "torcheval is required", + connect_requirement_message or torcheval_requirement_message, ) class EvaluationTestsOnConnect(EvaluationTestsMixin, unittest.TestCase): def setUp(self) -> None: diff --git a/python/pyspark/ml/tests/connect/test_connect_feature.py b/python/pyspark/ml/tests/connect/test_connect_feature.py index c1d02050097b2..879cbff6d0cc7 100644 --- a/python/pyspark/ml/tests/connect/test_connect_feature.py +++ b/python/pyspark/ml/tests/connect/test_connect_feature.py @@ -20,14 +20,7 @@ from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -have_sklearn = True -sklearn_requirement_message = None -try: - from sklearn.datasets import load_breast_cancer # noqa: F401 -except ImportError: - have_sklearn = False - sklearn_requirement_message = "No sklearn found" +from pyspark.testing.utils import have_sklearn, sklearn_requirement_message if should_test_connect: from pyspark.ml.tests.connect.test_legacy_mode_feature import FeatureTestsMixin diff --git a/python/pyspark/ml/tests/connect/test_connect_pipeline.py b/python/pyspark/ml/tests/connect/test_connect_pipeline.py index 7733af7631e92..f8576d0cb09da 100644 --- a/python/pyspark/ml/tests/connect/test_connect_pipeline.py +++ b/python/pyspark/ml/tests/connect/test_connect_pipeline.py @@ -21,14 +21,8 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message +from pyspark.testing.utils import have_torch, torch_requirement_message -torch_requirement_message = None -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - torch_requirement_message = "torch is required" if should_test_connect: from pyspark.ml.tests.connect.test_legacy_mode_pipeline import PipelineTestsMixin diff --git a/python/pyspark/ml/tests/connect/test_connect_tuning.py b/python/pyspark/ml/tests/connect/test_connect_tuning.py index fee7113e1ae5a..d737dd5767dbd 100644 --- a/python/pyspark/ml/tests/connect/test_connect_tuning.py +++ b/python/pyspark/ml/tests/connect/test_connect_tuning.py @@ -19,13 +19,10 @@ import unittest import os -from pyspark.ml.tests.connect.test_connect_classification import ( - have_torch, - torch_requirement_message, -) from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message +from pyspark.testing.utils import have_torch, torch_requirement_message if should_test_connect: from pyspark.ml.tests.connect.test_legacy_mode_tuning import CrossValidatorTestsMixin diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_classification.py b/python/pyspark/ml/tests/connect/test_legacy_mode_classification.py index dc2642a42d666..fdae31077002e 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_classification.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_classification.py @@ -24,14 +24,7 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -have_torch = True -torch_requirement_message = None -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - torch_requirement_message = "No torch found" +from pyspark.testing.utils import have_torch, torch_requirement_message if should_test_connect: from pyspark.ml.connect.classification import ( @@ -135,6 +128,8 @@ def test_multi_classes_logistic_regression(self): self._check_result(local_transform_result, expected_predictions, expected_probabilities) def test_save_load(self): + import torch + with tempfile.TemporaryDirectory(prefix="test_save_load") as tmp_dir: estimator = LORV2(maxIter=2, numTrainWorkers=2, learningRate=0.001) local_path = os.path.join(tmp_dir, "estimator") diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_evaluation.py b/python/pyspark/ml/tests/connect/test_legacy_mode_evaluation.py index 11c1f9aeee51d..3a5417dadf50a 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_evaluation.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_evaluation.py @@ -23,14 +23,7 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -have_torcheval = True -torcheval_requirement_message = None -try: - import torcheval # noqa: F401 -except ImportError: - have_torcheval = False - torcheval_requirement_message = "torcheval is required" +from pyspark.testing.utils import have_torcheval, torcheval_requirement_message if should_test_connect: from pyspark.ml.connect.evaluation import ( diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_feature.py b/python/pyspark/ml/tests/connect/test_legacy_mode_feature.py index d90e4a4315d5f..6812db778450a 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_feature.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_feature.py @@ -26,10 +26,7 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message -from pyspark.ml.tests.connect.test_connect_classification import ( - have_torch, - torch_requirement_message, -) +from pyspark.testing.utils import have_torch, torch_requirement_message if should_test_connect: from pyspark.ml.connect.feature import ( diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_pipeline.py b/python/pyspark/ml/tests/connect/test_legacy_mode_pipeline.py index 9165034718d79..8b19f5931d207 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_pipeline.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_pipeline.py @@ -24,10 +24,7 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession from pyspark.testing.connectutils import should_test_connect, connect_requirement_message -from pyspark.ml.tests.connect.test_connect_classification import ( - have_torch, - torch_requirement_message, -) +from pyspark.testing.utils import have_torch, torch_requirement_message if should_test_connect: from pyspark.ml.connect.feature import StandardScaler diff --git a/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py b/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py index 33cc39d3319db..06c3ad93d92d2 100644 --- a/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py +++ b/python/pyspark/ml/tests/connect/test_legacy_mode_tuning.py @@ -28,31 +28,14 @@ from pyspark.sql import SparkSession from pyspark.sql.functions import rand from pyspark.testing.connectutils import should_test_connect, connect_requirement_message - -have_sklearn = True -sklearn_requirement_message = None -try: - from sklearn.datasets import load_breast_cancer # noqa: F401 -except ImportError: - have_sklearn = False - sklearn_requirement_message = "No sklearn found" - -have_torch = True -torch_requirement_message = None -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - torch_requirement_message = "torch is required" - -have_torcheval = True -torcheval_requirement_message = None -try: - import torcheval # noqa: F401 -except ImportError: - have_torcheval = False - torcheval_requirement_message = "torcheval is required" - +from pyspark.testing.utils import ( + have_sklearn, + sklearn_requirement_message, + have_torch, + torch_requirement_message, + have_torcheval, + torcheval_requirement_message, +) if should_test_connect: import pandas as pd @@ -205,6 +188,8 @@ def _check_result(result_dataframe, expected_predictions, expected_probabilities ) def test_crossvalidator_on_pipeline(self): + from sklearn.datasets import load_breast_cancer + sk_dataset = load_breast_cancer() train_dataset = self.spark.createDataFrame( @@ -270,6 +255,8 @@ def _verify_cv_saved_params(instance, loaded_instance): sys.version_info > (3, 12), "SPARK-46078: Fails with dev torch with Python 3.12" ) def test_crossvalidator_with_fold_col(self): + from sklearn.datasets import load_breast_cancer + sk_dataset = load_breast_cancer() train_dataset = self.spark.createDataFrame( diff --git a/python/pyspark/ml/tests/connect/test_parity_torch_data_loader.py b/python/pyspark/ml/tests/connect/test_parity_torch_data_loader.py index 462fe3822141e..de05927138d4a 100644 --- a/python/pyspark/ml/tests/connect/test_parity_torch_data_loader.py +++ b/python/pyspark/ml/tests/connect/test_parity_torch_data_loader.py @@ -19,14 +19,7 @@ from pyspark.util import is_remote_only from pyspark.sql import SparkSession - -torch_requirement_message = None -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - torch_requirement_message = "torch is required" +from pyspark.testing.utils import have_torch, torch_requirement_message if not is_remote_only(): from pyspark.ml.torch.tests.test_data_loader import TorchDistributorDataLoaderUnitTests diff --git a/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py b/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py index e40303ae9ce21..3cd8abfc6e4ea 100644 --- a/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py +++ b/python/pyspark/ml/tests/connect/test_parity_torch_distributor.py @@ -19,16 +19,9 @@ import shutil import unittest -torch_requirement_message = None -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - torch_requirement_message = "torch is required" - from pyspark.util import is_remote_only from pyspark.sql import SparkSession +from pyspark.testing.utils import have_torch, torch_requirement_message if not is_remote_only(): from pyspark.ml.torch.tests.test_distributor import ( diff --git a/python/pyspark/ml/torch/tests/test_data_loader.py b/python/pyspark/ml/torch/tests/test_data_loader.py index 00f5f0a8c8d85..a47a5f163b687 100644 --- a/python/pyspark/ml/torch/tests/test_data_loader.py +++ b/python/pyspark/ml/torch/tests/test_data_loader.py @@ -17,23 +17,16 @@ import unittest -import numpy as np - -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - from pyspark.ml.torch.distributor import ( TorchDistributor, _get_spark_partition_data_loader, ) from pyspark.sql import SparkSession from pyspark.ml.linalg import Vectors +from pyspark.testing.utils import have_torch, torch_requirement_message -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchDistributorDataLoaderUnitTests(unittest.TestCase): def setUp(self) -> None: self.spark = ( @@ -46,6 +39,8 @@ def tearDown(self) -> None: self.spark.stop() def _check_data_loader_result_correctness(self, result, expected): + import numpy as np + assert len(result) == len(expected) for res_row, exp_row in zip(result, expected): diff --git a/python/pyspark/ml/torch/tests/test_distributor.py b/python/pyspark/ml/torch/tests/test_distributor.py index d16e60588482c..7b6a93afbff78 100644 --- a/python/pyspark/ml/torch/tests/test_distributor.py +++ b/python/pyspark/ml/torch/tests/test_distributor.py @@ -29,17 +29,11 @@ import unittest from unittest.mock import patch -have_torch = True -try: - import torch # noqa: F401 -except ImportError: - have_torch = False - from pyspark import SparkConf, SparkContext from pyspark.ml.torch.distributor import TorchDistributor, _get_gpus_owned from pyspark.ml.torch.torch_run_process_wrapper import clean_and_terminate, check_parent_alive from pyspark.sql import SparkSession -from pyspark.testing.utils import SPARK_HOME +from pyspark.testing.utils import SPARK_HOME, have_torch, torch_requirement_message @contextlib.contextmanager @@ -312,7 +306,7 @@ def test_create_torchrun_command(self) -> None: self.delete_env_vars(input_env_vars) -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchDistributorBaselineUnitTests(TorchDistributorBaselineUnitTestsMixin, unittest.TestCase): @classmethod def setUpClass(cls): @@ -409,7 +403,7 @@ def test_end_to_end_run_locally(self) -> None: self.assertEqual(output, "success" * 4096) -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchDistributorLocalUnitTests(TorchDistributorLocalUnitTestsMixin, unittest.TestCase): @classmethod def setUpClass(cls): @@ -431,7 +425,7 @@ def tearDownClass(cls): cls.spark.stop() -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchDistributorLocalUnitTestsII(TorchDistributorLocalUnitTestsMixin, unittest.TestCase): @classmethod def setUpClass(cls): @@ -502,7 +496,7 @@ def test_end_to_end_run_distributedly(self) -> None: self.assertEqual(output, "success" * 4096) -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchDistributorDistributedUnitTests( TorchDistributorDistributedUnitTestsMixin, unittest.TestCase ): @@ -549,7 +543,7 @@ def test_check_parent_alive(self, mock_clean_and_terminate: Callable) -> None: self.assertEqual(mock_clean_and_terminate.call_count, 0) -@unittest.skipIf(not have_torch, "torch is required") +@unittest.skipIf(not have_torch, torch_requirement_message) class TorchWrapperUnitTests(TorchWrapperUnitTestsMixin, unittest.TestCase): pass diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index 1773cdcf0a0a0..ca16628fc56f0 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -35,23 +35,6 @@ ) from itertools import zip_longest -have_scipy = False -have_numpy = False -try: - import scipy # noqa: F401 - - have_scipy = True -except ImportError: - # No SciPy, but that's okay, we'll skip those tests - pass -try: - import numpy as np # noqa: F401 - - have_numpy = True -except ImportError: - # No NumPy, but that's okay, we'll skip those tests - pass - from pyspark import SparkConf from pyspark.errors import PySparkAssertionError, PySparkException, PySparkTypeError from pyspark.errors.exceptions.captured import CapturedException @@ -68,6 +51,38 @@ SPARK_HOME = _find_spark_home() +def have_package(name: str) -> bool: + try: + import importlib + + importlib.import_module(name) + return True + except Exception: + return False + + +have_numpy = have_package("numpy") +numpy_requirement_message = None if have_numpy else "No module named 'numpy'" + +have_scipy = have_package("scipy") +scipy_requirement_message = None if have_scipy else "No module named 'scipy'" + +have_sklearn = have_package("sklearn") +sklearn_requirement_message = None if have_sklearn else "No module named 'sklearn'" + +have_torch = have_package("torch") +torch_requirement_message = None if have_torch else "No module named 'torch'" + +have_torcheval = have_package("torcheval") +torcheval_requirement_message = None if have_torcheval else "No module named 'torcheval'" + +have_deepspeed = have_package("deepspeed") +deepspeed_requirement_message = None if have_deepspeed else "No module named 'deepspeed'" + +have_plotly = have_package("plotly") +plotly_requirement_message = None if have_plotly else "No module named 'plotly'" + + def read_int(b): return struct.unpack("!i", b)[0] From 3bc374d945ff91cda78e64c1d63fe9a95f735ebf Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 21 Nov 2024 09:09:24 +0100 Subject: [PATCH 32/58] [SPARK-50333][SQL] Codegen Support for `CsvToStructs` (by Invoke & RuntimeReplaceable) ### What changes were proposed in this pull request? The pr aims to add `Codegen` Support for `CsvToStructs`(`from_csv`). ### Why are the changes needed? - improve codegen coverage. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA & Existed UT (eg: CsvFunctionsSuite#`*from_csv*`) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48873 from panbingkun/from_csv_codegen. Lead-authored-by: panbingkun Co-authored-by: panbingkun Signed-off-by: Max Gekk --- .../csv/CsvExpressionEvalUtils.scala | 70 ++++++++++++++- .../catalyst/expressions/csvExpressions.scala | 87 +++++-------------- .../explain-results/function_from_csv.explain | 2 +- 3 files changed, 93 insertions(+), 66 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala index abd0703fa7d70..a91e4ab13001b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csv/CsvExpressionEvalUtils.scala @@ -18,10 +18,78 @@ package org.apache.spark.sql.catalyst.expressions.csv import com.univocity.parsers.csv.CsvParser -import org.apache.spark.sql.catalyst.csv.{CSVInferSchema, CSVOptions} +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.{CSVInferSchema, CSVOptions, UnivocityParser} +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.util.{FailFastMode, FailureSafeParser, PermissiveMode} +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{DataType, NullType, StructType} import org.apache.spark.unsafe.types.UTF8String +/** + * The expression `CsvToStructs` will utilize the `Invoke` to call it, support codegen. + */ +case class CsvToStructsEvaluator( + options: Map[String, String], + nullableSchema: StructType, + nameOfCorruptRecord: String, + timeZoneId: Option[String], + requiredSchema: Option[StructType]) { + + // This converts parsed rows to the desired output by the given schema. + @transient + private lazy val converter = (rows: Iterator[InternalRow]) => { + if (!rows.hasNext) { + throw SparkException.internalError("Expected one row from CSV parser.") + } + val result = rows.next() + // CSV's parser produces one record only. + assert(!rows.hasNext) + result + } + + @transient + private lazy val parser = { + // 'lineSep' is a plan-wise option so we set a noncharacter, according to + // the unicode specification, which should not appear in Java's strings. + // See also SPARK-38955 and https://www.unicode.org/charts/PDF/UFFF0.pdf. + // scalastyle:off nonascii + val exprOptions = options ++ Map("lineSep" -> '\uFFFF'.toString) + // scalastyle:on nonascii + val parsedOptions = new CSVOptions( + exprOptions, + columnPruning = true, + defaultTimeZoneId = timeZoneId.get, + defaultColumnNameOfCorruptRecord = nameOfCorruptRecord) + val mode = parsedOptions.parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw QueryCompilationErrors.parseModeUnsupportedError("from_csv", mode) + } + ExprUtils.verifyColumnNameOfCorruptRecord( + nullableSchema, + parsedOptions.columnNameOfCorruptRecord) + + val actualSchema = + StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val actualRequiredSchema = + StructType(requiredSchema.map(_.asNullable).getOrElse(nullableSchema) + .filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val rawParser = new UnivocityParser(actualSchema, + actualRequiredSchema, + parsedOptions) + new FailureSafeParser[String]( + input => rawParser.parse(input), + mode, + nullableSchema, + parsedOptions.columnNameOfCorruptRecord) + } + + final def evaluate(csv: UTF8String): InternalRow = { + converter(parser.parse(csv.toString)) + } +} + case class SchemaOfCsvEvaluator(options: Map[String, String]) { @transient diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index e9cdc184e55a3..02e5488835c91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -19,17 +19,16 @@ package org.apache.spark.sql.catalyst.expressions import java.io.CharArrayWriter -import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} import org.apache.spark.sql.catalyst.csv._ -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} -import org.apache.spark.sql.catalyst.expressions.csv.SchemaOfCsvEvaluator +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.csv.{CsvToStructsEvaluator, SchemaOfCsvEvaluator} import org.apache.spark.sql.catalyst.expressions.objects.Invoke -import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.trees.TreePattern.{RUNTIME_REPLACEABLE, TreePattern} import org.apache.spark.sql.catalyst.util.TypeUtils._ -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.errors.QueryErrorsBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types._ @@ -58,15 +57,17 @@ case class CsvToStructs( timeZoneId: Option[String] = None, requiredSchema: Option[StructType] = None) extends UnaryExpression - with TimeZoneAwareExpression - with CodegenFallback - with ExpectsInputTypes { - override def nullIntolerant: Boolean = true + with RuntimeReplaceable + with ExpectsInputTypes + with TimeZoneAwareExpression { + override def nullable: Boolean = child.nullable + override def nodePatternsInternal(): Seq[TreePattern] = Seq(RUNTIME_REPLACEABLE) + // The CSV input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. - val nullableSchema: StructType = schema.asNullable + private val nullableSchema: StructType = schema.asNullable // Used in `FunctionRegistry` def this(child: Expression, schema: Expression, options: Map[String, String]) = @@ -85,55 +86,7 @@ case class CsvToStructs( child = child, timeZoneId = None) - // This converts parsed rows to the desired output by the given schema. - @transient - lazy val converter = (rows: Iterator[InternalRow]) => { - if (rows.hasNext) { - val result = rows.next() - // CSV's parser produces one record only. - assert(!rows.hasNext) - result - } else { - throw SparkException.internalError("Expected one row from CSV parser.") - } - } - - val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - - @transient lazy val parser = { - // 'lineSep' is a plan-wise option so we set a noncharacter, according to - // the unicode specification, which should not appear in Java's strings. - // See also SPARK-38955 and https://www.unicode.org/charts/PDF/UFFF0.pdf. - // scalastyle:off nonascii - val exprOptions = options ++ Map("lineSep" -> '\uFFFF'.toString) - // scalastyle:on nonascii - val parsedOptions = new CSVOptions( - exprOptions, - columnPruning = true, - defaultTimeZoneId = timeZoneId.get, - defaultColumnNameOfCorruptRecord = nameOfCorruptRecord) - val mode = parsedOptions.parseMode - if (mode != PermissiveMode && mode != FailFastMode) { - throw QueryCompilationErrors.parseModeUnsupportedError("from_csv", mode) - } - ExprUtils.verifyColumnNameOfCorruptRecord( - nullableSchema, - parsedOptions.columnNameOfCorruptRecord) - - val actualSchema = - StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val actualRequiredSchema = - StructType(requiredSchema.map(_.asNullable).getOrElse(nullableSchema) - .filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val rawParser = new UnivocityParser(actualSchema, - actualRequiredSchema, - parsedOptions) - new FailureSafeParser[String]( - input => rawParser.parse(input), - mode, - nullableSchema, - parsedOptions.columnNameOfCorruptRecord) - } + private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) override def dataType: DataType = requiredSchema.getOrElse(schema).asNullable @@ -141,15 +94,21 @@ case class CsvToStructs( copy(timeZoneId = Option(timeZoneId)) } - override def nullSafeEval(input: Any): Any = { - val csv = input.asInstanceOf[UTF8String].toString - converter(parser.parse(csv)) - } - override def inputTypes: Seq[AbstractDataType] = StringTypeWithCollation :: Nil override def prettyName: String = "from_csv" + @transient + private lazy val evaluator: CsvToStructsEvaluator = CsvToStructsEvaluator( + options, nullableSchema, nameOfCorruptRecord, timeZoneId, requiredSchema) + + override def replacement: Expression = Invoke( + Literal.create(evaluator, ObjectType(classOf[CsvToStructsEvaluator])), + "evaluate", + dataType, + Seq(child), + Seq(child.dataType)) + override protected def withNewChildInternal(newChild: Expression): CsvToStructs = copy(child = newChild) } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain index 89e03c8188232..ef87c18948b23 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_from_csv.explain @@ -1,2 +1,2 @@ -Project [from_csv(StructField(id,LongType,true), StructField(a,IntegerType,true), StructField(b,DoubleType,true), (mode,FAILFAST), g#0, Some(America/Los_Angeles), None) AS from_csv(g)#0] +Project [invoke(CsvToStructsEvaluator(Map(mode -> FAILFAST),StructType(StructField(id,LongType,true),StructField(a,IntegerType,true),StructField(b,DoubleType,true)),_corrupt_record,Some(America/Los_Angeles),None).evaluate(g#0)) AS from_csv(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] From 95faa0283e18343adba4b39083829d3c3724c035 Mon Sep 17 00:00:00 2001 From: Mark Andreev Date: Thu, 21 Nov 2024 09:14:06 +0100 Subject: [PATCH 33/58] [SPARK-49490][SQL] Add benchmarks for initCap ### What changes were proposed in this pull request? Add benchmarks for all codepaths of initCap, namely, paths that call: - execBinaryICU - execBinary - execLowercase - execICU ### Why are the changes needed? Requested by jira ticket SPARK-49490. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The benchmark was tested locally by performing a manual run. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48501 from mrk-andreev/SPARK-49490. Authored-by: Mark Andreev Signed-off-by: Max Gekk --- .../CollationBenchmark-jdk21-results.txt | 106 ++++++++++++------ .../benchmarks/CollationBenchmark-results.txt | 106 ++++++++++++------ ...llationNonASCIIBenchmark-jdk21-results.txt | 106 ++++++++++++------ .../CollationNonASCIIBenchmark-results.txt | 106 ++++++++++++------ .../benchmark/CollationBenchmark.scala | 44 ++++++++ 5 files changed, 324 insertions(+), 144 deletions(-) diff --git a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt index e31b45b48f8f0..88db9ebfa1e34 100644 --- a/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CollationBenchmark-jdk21-results.txt @@ -1,54 +1,88 @@ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1353 1357 5 0.1 13532.2 1.0X -UTF8_LCASE 2601 2602 2 0.0 26008.0 1.9X -UNICODE 16745 16756 16 0.0 167450.9 12.4X -UNICODE_CI 16590 16627 52 0.0 165904.8 12.3X +UTF8_BINARY 1193 1194 1 0.1 11929.0 1.0X +UTF8_LCASE 2717 2721 6 0.0 27168.5 2.3X +UNICODE 17991 17993 2 0.0 179913.6 15.1X +UNICODE_CI 17837 17842 7 0.0 178369.9 15.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1746 1746 0 0.1 17462.6 1.0X -UTF8_LCASE 2629 2630 1 0.0 26294.8 1.5X -UNICODE 16744 16744 0 0.0 167438.6 9.6X -UNICODE_CI 16518 16521 4 0.0 165180.2 9.5X +UTF8_BINARY 1523 1523 0 0.1 15233.9 1.0X +UTF8_LCASE 2441 2441 0 0.0 24407.9 1.6X +UNICODE 17875 17884 13 0.0 178749.6 11.7X +UNICODE_CI 17701 17703 2 0.0 177013.8 11.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2808 2808 1 0.0 28076.2 1.0X -UTF8_LCASE 5409 5410 0 0.0 54093.0 1.9X -UNICODE 67930 67957 38 0.0 679296.7 24.2X -UNICODE_CI 56004 56005 1 0.0 560044.2 19.9X +UTF8_BINARY 2660 2666 9 0.0 26601.1 1.0X +UTF8_LCASE 5013 5016 3 0.0 50134.0 1.9X +UNICODE 75622 75623 1 0.0 756217.3 28.4X +UNICODE_CI 63036 63042 9 0.0 630360.9 23.7X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 1612 1614 2 0.1 16118.8 1.0X -UTF8_LCASE 14509 14526 23 0.0 145092.7 9.0X -UNICODE 308136 308631 700 0.0 3081364.6 191.2X -UNICODE_CI 314612 314846 330 0.0 3146120.0 195.2X +UTF8_BINARY 2121 2122 0 0.0 21214.2 1.0X +UTF8_LCASE 27635 27636 1 0.0 276347.7 13.0X +UNICODE 523746 524012 376 0.0 5237460.5 246.9X +UNICODE_CI 520134 520227 131 0.0 5201343.3 245.2X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 1913 1914 1 0.1 19131.3 1.0X -UTF8_LCASE 9785 9788 5 0.0 97847.7 5.1X -UNICODE 311517 311580 89 0.0 3115167.2 162.8X -UNICODE_CI 316517 316660 201 0.0 3165173.7 165.4X +UTF8_BINARY 2767 2769 4 0.0 27666.3 1.0X +UTF8_LCASE 26861 26861 1 0.0 268606.4 9.7X +UNICODE 518540 518815 389 0.0 5185401.3 187.4X +UNICODE_CI 521156 521261 148 0.0 5211559.5 188.4X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 1891 1891 0 0.1 18912.1 1.0X -UTF8_LCASE 10089 10093 5 0.0 100893.6 5.3X -UNICODE 336905 336931 36 0.0 3369051.8 178.1X -UNICODE_CI 339944 340585 907 0.0 3399439.0 179.7X +UTF8_BINARY 2919 2921 3 0.0 29190.2 1.0X +UTF8_LCASE 26862 26862 1 0.0 268618.0 9.2X +UNICODE 504534 504927 556 0.0 5045340.3 172.8X +UNICODE_CI 506542 506565 32 0.0 5065423.0 173.5X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------- +UNICODE 419 425 5 0.2 4189.2 1.0X +UNICODE_CI 416 426 6 0.2 4163.2 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 575 576 0 0.2 5754.0 1.0X +UTF8_LCASE 575 576 1 0.2 5747.8 1.0X +UNICODE 576 576 0 0.2 5761.5 1.0X +UNICODE_CI 576 578 2 0.2 5758.0 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +----------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 159 159 1 0.6 1587.6 1.0X +UTF8_LCASE 159 159 0 0.6 1586.6 1.0X +UNICODE 158 159 1 0.6 1584.9 1.0X +UNICODE_CI 159 160 1 0.6 1586.1 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 397 405 5 0.3 3974.4 1.0X +UTF8_LCASE 401 405 5 0.2 4009.5 1.0X +UNICODE 395 399 3 0.3 3953.9 1.0X +UNICODE_CI 395 400 3 0.3 3952.0 1.0X diff --git a/sql/core/benchmarks/CollationBenchmark-results.txt b/sql/core/benchmarks/CollationBenchmark-results.txt index eb8ae040a46f6..8402a2db6d869 100644 --- a/sql/core/benchmarks/CollationBenchmark-results.txt +++ b/sql/core/benchmarks/CollationBenchmark-results.txt @@ -1,54 +1,88 @@ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1373 1373 0 0.1 13730.8 1.0X -UTF8_LCASE 3311 3311 0 0.0 33106.6 2.4X -UNICODE 19067 19100 46 0.0 190672.9 13.9X -UNICODE_CI 18704 18795 129 0.0 187040.2 13.6X +UTF8_BINARY 1223 1224 1 0.1 12231.5 1.0X +UTF8_LCASE 3280 3281 1 0.0 32803.3 2.7X +UNICODE 17207 17207 0 0.0 172065.7 14.1X +UNICODE_CI 16560 16565 7 0.0 165604.3 13.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 1706 1708 3 0.1 17060.4 1.0X -UTF8_LCASE 3958 3965 10 0.0 39575.4 2.3X -UNICODE 18831 18865 48 0.0 188311.2 11.0X -UNICODE_CI 18818 18825 9 0.0 188181.7 11.0X +UTF8_BINARY 1656 1657 0 0.1 16564.0 1.0X +UTF8_LCASE 3320 3321 0 0.0 33203.0 2.0X +UNICODE 16392 16393 2 0.0 163921.3 9.9X +UNICODE_CI 16314 16319 6 0.0 163143.3 9.8X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 3092 3093 1 0.0 30918.5 1.0X -UTF8_LCASE 6273 6289 23 0.0 62734.3 2.0X -UNICODE 66953 66962 13 0.0 669525.2 21.7X -UNICODE_CI 53934 53946 17 0.0 539338.7 17.4X +UTF8_BINARY 2812 2813 1 0.0 28119.0 1.0X +UTF8_LCASE 5682 5685 4 0.0 56823.2 2.0X +UNICODE 71678 71685 10 0.0 716777.4 25.5X +UNICODE_CI 60660 60670 15 0.0 606597.4 21.6X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 1643 1644 1 0.1 16431.2 1.0X -UTF8_LCASE 17241 17273 45 0.0 172411.1 10.5X -UNICODE 304878 307207 3294 0.0 3048780.8 185.5X -UNICODE_CI 317341 320620 4637 0.0 3173412.3 193.1X +UTF8_BINARY 2528 2528 1 0.0 25276.8 1.0X +UTF8_LCASE 28034 28050 24 0.0 280335.5 11.1X +UNICODE 521518 521690 242 0.0 5215184.7 206.3X +UNICODE_CI 508188 508312 176 0.0 5081880.5 201.0X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 1973 1977 6 0.1 19726.2 1.0X -UTF8_LCASE 17070 17119 70 0.0 170697.7 8.7X -UNICODE 306091 306797 999 0.0 3060911.4 155.2X -UNICODE_CI 306558 307812 1774 0.0 3065581.4 155.4X +UTF8_BINARY 2772 2774 4 0.0 27715.0 1.0X +UTF8_LCASE 27387 27390 4 0.0 273872.8 9.9X +UNICODE 501025 501076 72 0.0 5010249.5 180.8X +UNICODE_CI 506654 506666 16 0.0 5066544.6 182.8X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 2064 2064 0 0.0 20640.6 1.0X -UTF8_LCASE 16883 16899 23 0.0 168829.3 8.2X -UNICODE 309882 310702 1160 0.0 3098819.7 150.1X -UNICODE_CI 313599 314798 1695 0.0 3135994.6 151.9X +UTF8_BINARY 2886 2888 3 0.0 28858.9 1.0X +UTF8_LCASE 27433 27445 17 0.0 274326.2 9.5X +UNICODE 501068 501186 168 0.0 5010676.2 173.6X +UNICODE_CI 506619 506655 52 0.0 5066185.6 175.6X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------- +UNICODE 407 411 4 0.2 4065.4 1.0X +UNICODE_CI 419 423 3 0.2 4194.1 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 564 565 2 0.2 5639.2 1.0X +UTF8_LCASE 563 563 0 0.2 5629.0 1.0X +UNICODE 563 565 2 0.2 5634.3 1.0X +UNICODE_CI 564 564 0 0.2 5640.9 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +----------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 165 166 1 0.6 1647.3 1.0X +UTF8_LCASE 165 165 1 0.6 1646.7 1.0X +UNICODE 165 165 1 0.6 1646.5 1.0X +UNICODE_CI 165 166 1 0.6 1648.7 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 391 399 6 0.3 3912.1 1.0X +UTF8_LCASE 389 399 7 0.3 3894.2 1.0X +UNICODE 383 391 6 0.3 3828.6 1.0X +UNICODE_CI 383 387 2 0.3 3833.0 1.0X diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt index 58ceaecb795d3..4da64ade11d68 100644 --- a/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-jdk21-results.txt @@ -1,54 +1,88 @@ -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 170 171 2 0.2 4260.8 1.0X -UTF8_LCASE 7006 7009 5 0.0 175152.5 41.1X -UNICODE 5152 5153 2 0.0 128798.5 30.2X -UNICODE_CI 5307 5319 18 0.0 132666.4 31.1X +UTF8_BINARY 156 156 0 0.3 3887.8 1.0X +UTF8_LCASE 9717 9729 18 0.0 242914.7 62.5X +UNICODE 5026 5027 2 0.0 125640.1 32.3X +UNICODE_CI 4969 4972 4 0.0 124224.9 32.0X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 310 311 1 0.1 7748.1 1.0X -UTF8_LCASE 6971 6977 8 0.0 174277.9 22.5X -UNICODE 5788 5794 9 0.0 144708.3 18.7X -UNICODE_CI 5710 5715 7 0.0 142754.1 18.4X +UTF8_BINARY 279 279 0 0.1 6969.5 1.0X +UTF8_LCASE 9624 9628 5 0.0 240611.6 34.5X +UNICODE 5243 5244 0 0.0 131080.1 18.8X +UNICODE_CI 5173 5173 0 0.0 129322.8 18.6X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 379 380 1 0.1 9479.8 1.0X -UTF8_LCASE 3541 3547 7 0.0 88536.4 9.3X -UNICODE 15014 15030 22 0.0 375356.2 39.6X -UNICODE_CI 12226 12231 6 0.0 305661.7 32.2X +UTF8_BINARY 383 383 0 0.1 9576.7 1.0X +UTF8_LCASE 4927 4931 6 0.0 123170.3 12.9X +UNICODE 17244 17261 24 0.0 431096.6 45.0X +UNICODE_CI 12968 12970 3 0.0 324194.1 33.9X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 315 315 1 0.1 7864.6 1.0X -UTF8_LCASE 7995 8005 14 0.0 199880.8 25.4X -UNICODE 58712 58801 125 0.0 1467803.8 186.6X -UNICODE_CI 58777 58784 9 0.0 1469433.1 186.8X +UTF8_BINARY 535 536 2 0.1 13371.6 1.0X +UTF8_LCASE 9479 9480 2 0.0 236964.5 17.7X +UNICODE 93629 93676 66 0.0 2340726.5 175.1X +UNICODE_CI 93222 93309 124 0.0 2330541.2 174.3X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 284 284 1 0.1 7093.4 1.0X -UTF8_LCASE 5169 5171 4 0.0 129215.5 18.2X -UNICODE 57857 57897 57 0.0 1446425.3 203.9X -UNICODE_CI 58803 58826 32 0.0 1470086.9 207.2X +UTF8_BINARY 430 431 1 0.1 10755.8 1.0X +UTF8_LCASE 6550 6551 2 0.0 163753.7 15.2X +UNICODE 87435 87467 45 0.0 2185886.8 203.2X +UNICODE_CI 90113 90255 201 0.0 2252836.0 209.5X -OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 284 284 0 0.1 7090.7 1.0X -UTF8_LCASE 5187 5187 1 0.0 129665.9 18.3X -UNICODE 64562 64565 5 0.0 1614041.8 227.6X -UNICODE_CI 63633 63686 75 0.0 1590826.2 224.4X +UTF8_BINARY 455 456 2 0.1 11369.5 1.0X +UTF8_LCASE 7108 7115 9 0.0 177705.2 15.6X +UNICODE 101835 101866 43 0.0 2545883.9 223.9X +UNICODE_CI 100962 101026 91 0.0 2524045.2 222.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------- +UNICODE 254 255 1 0.2 6346.5 1.0X +UNICODE_CI 254 254 0 0.2 6348.1 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 322 323 1 0.1 8046.3 1.0X +UTF8_LCASE 322 324 2 0.1 8059.0 1.0X +UNICODE 322 323 1 0.1 8050.7 1.0X +UNICODE_CI 322 325 4 0.1 8062.4 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +----------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 119 120 1 0.3 2972.1 1.0X +UTF8_LCASE 119 120 1 0.3 2971.9 1.0X +UNICODE 119 120 1 0.3 2970.3 1.0X +UNICODE_CI 119 120 1 0.3 2968.6 1.0X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 254 255 1 0.2 6345.2 1.0X +UTF8_LCASE 254 255 0 0.2 6351.8 1.0X +UNICODE 254 255 0 0.2 6352.9 1.0X +UNICODE_CI 254 254 0 0.2 6341.2 1.0X diff --git a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt index 2920ffbeac6a3..fba59f3893e22 100644 --- a/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt +++ b/sql/core/benchmarks/CollationNonASCIIBenchmark-results.txt @@ -1,54 +1,88 @@ -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - equalsFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time -------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 133 133 1 0.3 3314.1 1.0X -UTF8_LCASE 7149 7159 15 0.0 178721.0 53.9X -UNICODE 5693 5699 8 0.0 142326.0 42.9X -UNICODE_CI 6228 6231 5 0.0 155690.1 47.0X +UTF8_BINARY 125 126 1 0.3 3128.6 1.0X +UTF8_LCASE 10335 10345 14 0.0 258377.4 82.6X +UNICODE 5604 5610 8 0.0 140110.8 44.8X +UNICODE_CI 5570 5577 9 0.0 139252.7 44.5X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - compareFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time --------------------------------------------------------------------------------------------------------------------------- -UTF8_BINARY 472 475 3 0.1 11799.9 1.0X -UTF8_LCASE 7398 7423 37 0.0 184940.1 15.7X -UNICODE 6079 6083 5 0.0 151983.2 12.9X -UNICODE_CI 6032 6034 2 0.0 150811.6 12.8X +UTF8_BINARY 293 294 2 0.1 7326.8 1.0X +UTF8_LCASE 10035 10035 1 0.0 250865.2 34.2X +UNICODE 5578 5580 3 0.0 139455.8 19.0X +UNICODE_CI 5539 5541 2 0.0 138483.8 18.9X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - hashFunction: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 408 411 2 0.1 10203.7 1.0X -UTF8_LCASE 3642 3644 4 0.0 91040.2 8.9X -UNICODE 13471 13477 9 0.0 336766.1 33.0X -UNICODE_CI 11242 11249 10 0.0 281047.3 27.5X +UTF8_BINARY 388 388 0 0.1 9699.6 1.0X +UTF8_LCASE 4965 4967 3 0.0 124121.3 12.8X +UNICODE 15750 15753 5 0.0 393740.9 40.6X +UNICODE_CI 12509 12511 2 0.0 312735.5 32.2X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - contains: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 330 331 1 0.1 8249.5 1.0X -UTF8_LCASE 8380 8388 12 0.0 209490.6 25.4X -UNICODE 59720 59750 43 0.0 1493005.1 181.0X -UNICODE_CI 57747 57748 2 0.0 1443681.8 175.0X +UTF8_BINARY 421 422 2 0.1 10512.9 1.0X +UTF8_LCASE 10793 10796 5 0.0 269819.0 25.7X +UNICODE 94324 94330 9 0.0 2358090.9 224.3X +UNICODE_CI 91647 91748 143 0.0 2291174.6 217.9X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - startsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 312 312 1 0.1 7798.9 1.0X -UTF8_LCASE 5300 5300 1 0.0 132491.4 17.0X -UNICODE 57388 57515 179 0.0 1434709.7 184.0X -UNICODE_CI 57642 57764 171 0.0 1441059.6 184.8X +UTF8_BINARY 452 453 0 0.1 11307.9 1.0X +UTF8_LCASE 6871 6872 2 0.0 171782.0 15.2X +UNICODE 90881 90924 60 0.0 2272034.5 200.9X +UNICODE_CI 91333 91363 42 0.0 2283331.3 201.9X -OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure -AMD EPYC 7763 64-Core Processor +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz collation unit benchmarks - endsWith: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time ------------------------------------------------------------------------------------------------------------------------ -UTF8_BINARY 292 293 0 0.1 7311.6 1.0X -UTF8_LCASE 5375 5375 1 0.0 134375.2 18.4X -UNICODE 62773 63030 363 0.0 1569337.4 214.6X -UNICODE_CI 61268 61382 161 0.0 1531693.6 209.5X +UTF8_BINARY 451 452 2 0.1 11268.1 1.0X +UTF8_LCASE 6685 6686 2 0.0 167120.8 14.8X +UNICODE 99387 99484 138 0.0 2484672.5 220.5X +UNICODE_CI 98525 98597 101 0.0 2463132.9 218.6X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------- +UNICODE 231 232 0 0.2 5784.5 1.0X +UNICODE_CI 231 232 1 0.2 5780.4 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinaryICU: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 312 314 1 0.1 7811.2 1.0X +UTF8_LCASE 313 314 2 0.1 7822.9 1.0X +UNICODE 313 314 1 0.1 7815.5 1.0X +UNICODE_CI 313 315 4 0.1 7825.7 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execBinary: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +----------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 132 133 0 0.3 3302.0 1.0X +UTF8_LCASE 132 132 0 0.3 3297.5 1.0X +UNICODE 132 133 1 0.3 3296.9 1.0X +UNICODE_CI 132 132 0 0.3 3298.1 1.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.8.0-1018-aws +Intel(R) Xeon(R) Platinum 8252C CPU @ 3.80GHz +collation unit benchmarks - initCap using impl execLowercase: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative time +-------------------------------------------------------------------------------------------------------------------------------------------- +UTF8_BINARY 231 231 0 0.2 5770.4 1.0X +UTF8_LCASE 231 232 1 0.2 5776.4 1.0X +UNICODE 231 231 0 0.2 5767.5 1.0X +UNICODE_CI 231 232 1 0.2 5770.2 1.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala index 21a5a4a979eb0..37297f6aa94c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala @@ -185,6 +185,48 @@ abstract class CollationBenchmarkBase extends BenchmarkBase { } benchmark.run(relativeTime = true) } + + def benchmarkInitCap( + collationTypes: Seq[String], + utf8Strings: Seq[UTF8String]): Unit = { + type CollationId = Int + type InitCapEstimator = (UTF8String, CollationId) => Unit + def skipCollationTypeFilter: Any => Boolean = _ => true + def createBenchmark( + implName: String, + impl: InitCapEstimator, + collationTypeFilter: String => Boolean): Unit = { + val benchmark = new Benchmark( + s"collation unit benchmarks - initCap using impl $implName", + utf8Strings.size * 10, + warmupTime = 10.seconds, + output = output) + collationTypes.filter(collationTypeFilter).foreach { collationType => { + val collationId = CollationFactory.collationNameToId(collationType) + benchmark.addCase(collationType) { _ => + utf8Strings.foreach { s => + impl(s.repeat(1_000), collationId) + } + } + } + } + benchmark.run(relativeTime = true) + } + + createBenchmark( + "execICU", + (s, collationId) => CollationSupport.InitCap.execICU(s, collationId), + collationType => CollationFactory.fetchCollation(collationType).collator != null) + createBenchmark( + "execBinaryICU", + (s, _) => CollationSupport.InitCap.execBinaryICU(s), skipCollationTypeFilter) + createBenchmark( + "execBinary", + (s, _) => CollationSupport.InitCap.execBinary(s), skipCollationTypeFilter) + createBenchmark( + "execLowercase", + (s, _) => CollationSupport.InitCap.execLowercase(s), skipCollationTypeFilter) + } } /** @@ -219,6 +261,7 @@ object CollationBenchmark extends CollationBenchmarkBase { benchmarkContains(collationTypes, inputs) benchmarkStartsWith(collationTypes, inputs) benchmarkEndsWith(collationTypes, inputs) + benchmarkInitCap(collationTypes, inputs) } } @@ -248,5 +291,6 @@ object CollationNonASCIIBenchmark extends CollationBenchmarkBase { benchmarkContains(collationTypes, inputs) benchmarkStartsWith(collationTypes, inputs) benchmarkEndsWith(collationTypes, inputs) + benchmarkInitCap(collationTypes, inputs) } } From ee21e6b07a0d30cbdf78a2dd6bfe43d8fc23d518 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Thu, 21 Nov 2024 17:29:18 +0900 Subject: [PATCH 34/58] [SPARK-50113][CONNECT][PYTHON][TESTS] Add `@remote_only` to check the APIs that only supported with Spark Connect ### What changes were proposed in this pull request? This PR proposes to add `remote_only` to check the APIs that only supported with Spark Connect ### Why are the changes needed? The current compatibility check cannot capture the missing methods that only supported with Spark Connect ### Does this PR introduce _any_ user-facing change? No, it's test-only ### How was this patch tested? Updated the existing UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #48651 from itholic/SPARK-50113. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- python/pyspark/sql/session.py | 21 ++++++++- .../sql/tests/test_connect_compatibility.py | 43 +++++++++++++++---- python/pyspark/sql/utils.py | 16 +++++++ python/pyspark/util.py | 19 +++++--- 4 files changed, 84 insertions(+), 15 deletions(-) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index ef8750b6e72d4..7231d6c10b0b6 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -63,7 +63,12 @@ _from_numpy_type, ) from pyspark.errors.exceptions.captured import install_exception_handler -from pyspark.sql.utils import is_timestamp_ntz_preferred, to_str, try_remote_session_classmethod +from pyspark.sql.utils import ( + is_timestamp_ntz_preferred, + to_str, + try_remote_session_classmethod, + remote_only, +) from pyspark.errors import PySparkValueError, PySparkTypeError, PySparkRuntimeError if TYPE_CHECKING: @@ -550,6 +555,7 @@ def getOrCreate(self) -> "SparkSession": return session # Spark Connect-specific API + @remote_only def create(self) -> "SparkSession": """Creates a new SparkSession. Can only be used in the context of Spark Connect and will throw an exception otherwise. @@ -2067,6 +2073,7 @@ def __exit__( # SparkConnect-specific API @property + @remote_only def client(self) -> "SparkConnectClient": """ Gives access to the Spark Connect client. In normal cases this is not necessary to be used @@ -2090,6 +2097,7 @@ def client(self) -> "SparkConnectClient": messageParameters={"feature": "SparkSession.client"}, ) + @remote_only def addArtifacts( self, *path: str, pyfile: bool = False, archive: bool = False, file: bool = False ) -> None: @@ -2125,6 +2133,7 @@ def addArtifacts( addArtifact = addArtifacts + @remote_only def registerProgressHandler(self, handler: "ProgressHandler") -> None: """ Register a progress handler to be called when a progress update is received from the server. @@ -2153,6 +2162,7 @@ def registerProgressHandler(self, handler: "ProgressHandler") -> None: messageParameters={"feature": "SparkSession.registerProgressHandler"}, ) + @remote_only def removeProgressHandler(self, handler: "ProgressHandler") -> None: """ Remove a progress handler that was previously registered. @@ -2169,6 +2179,7 @@ def removeProgressHandler(self, handler: "ProgressHandler") -> None: messageParameters={"feature": "SparkSession.removeProgressHandler"}, ) + @remote_only def clearProgressHandlers(self) -> None: """ Clear all registered progress handlers. @@ -2180,6 +2191,7 @@ def clearProgressHandlers(self) -> None: messageParameters={"feature": "SparkSession.clearProgressHandlers"}, ) + @remote_only def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: """ Copy file from local to cloud storage file system. @@ -2208,6 +2220,7 @@ def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: messageParameters={"feature": "SparkSession.copyFromLocalToFs"}, ) + @remote_only def interruptAll(self) -> List[str]: """ Interrupt all operations of this session currently running on the connected server. @@ -2228,6 +2241,7 @@ def interruptAll(self) -> List[str]: messageParameters={"feature": "SparkSession.interruptAll"}, ) + @remote_only def interruptTag(self, tag: str) -> List[str]: """ Interrupt all operations of this session with the given operation tag. @@ -2248,6 +2262,7 @@ def interruptTag(self, tag: str) -> List[str]: messageParameters={"feature": "SparkSession.interruptTag"}, ) + @remote_only def interruptOperation(self, op_id: str) -> List[str]: """ Interrupt an operation of this session with the given operationId. @@ -2268,6 +2283,7 @@ def interruptOperation(self, op_id: str) -> List[str]: messageParameters={"feature": "SparkSession.interruptOperation"}, ) + @remote_only def addTag(self, tag: str) -> None: """ Add a tag to be assigned to all the operations started by this thread in this session. @@ -2292,6 +2308,7 @@ def addTag(self, tag: str) -> None: messageParameters={"feature": "SparkSession.addTag"}, ) + @remote_only def removeTag(self, tag: str) -> None: """ Remove a tag previously added to be assigned to all the operations started by this thread in @@ -2309,6 +2326,7 @@ def removeTag(self, tag: str) -> None: messageParameters={"feature": "SparkSession.removeTag"}, ) + @remote_only def getTags(self) -> Set[str]: """ Get the tags that are currently set to be assigned to all the operations started by this @@ -2326,6 +2344,7 @@ def getTags(self) -> Set[str]: messageParameters={"feature": "SparkSession.getTags"}, ) + @remote_only def clearTags(self) -> None: """ Clear the current thread's operation tags. diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index e20188e8da6f7..3d74e796cd7a0 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -64,12 +64,16 @@ class ConnectCompatibilityTestsMixin: def get_public_methods(self, cls): """Get public methods of a class.""" - return { - name: method - for name, method in inspect.getmembers(cls) - if (inspect.isfunction(method) or isinstance(method, functools._lru_cache_wrapper)) - and not name.startswith("_") - } + methods = {} + for name, method in inspect.getmembers(cls): + if ( + inspect.isfunction(method) or isinstance(method, functools._lru_cache_wrapper) + ) and not name.startswith("_"): + if getattr(method, "_remote_only", False): + methods[name] = None + else: + methods[name] = method + return methods def get_public_properties(self, cls): """Get public properties of a class.""" @@ -88,6 +92,10 @@ def compare_method_signatures(self, classic_cls, connect_cls, cls_name): common_methods = set(classic_methods.keys()) & set(connect_methods.keys()) for method in common_methods: + # Skip non-callable, Spark Connect-specific methods + if classic_methods[method] is None or connect_methods[method] is None: + continue + classic_signature = inspect.signature(classic_methods[method]) connect_signature = inspect.signature(connect_methods[method]) @@ -145,7 +153,11 @@ def check_missing_methods( connect_methods = self.get_public_methods(connect_cls) # Identify missing methods - classic_only_methods = set(classic_methods.keys()) - set(connect_methods.keys()) + classic_only_methods = { + name + for name, method in classic_methods.items() + if name not in connect_methods or method is None + } connect_only_methods = set(connect_methods.keys()) - set(classic_methods.keys()) # Compare the actual missing methods with the expected ones @@ -249,7 +261,22 @@ def test_spark_session_compatibility(self): """Test SparkSession compatibility between classic and connect.""" expected_missing_connect_properties = {"sparkContext"} expected_missing_classic_properties = {"is_stopped", "session_id"} - expected_missing_connect_methods = {"newSession"} + expected_missing_connect_methods = { + "addArtifact", + "addArtifacts", + "addTag", + "clearProgressHandlers", + "clearTags", + "copyFromLocalToFs", + "getTags", + "interruptAll", + "interruptOperation", + "interruptTag", + "newSession", + "registerProgressHandler", + "removeProgressHandler", + "removeTag", + } expected_missing_classic_methods = set() self.check_compatibility( ClassicSparkSession, diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index cb59e0c7b4391..3cacc5b9d021a 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -458,3 +458,19 @@ def linspace(start: float, stop: float, num: int) -> Sequence[float]: return [float(start)] step = (float(stop) - float(start)) / (num - 1) return [start + step * i for i in range(num)] + + +def remote_only(func: Union[Callable, property]) -> Union[Callable, property]: + """ + Decorator to mark a function or method as only available in Spark Connect. + + This decorator allows for easy identification of Spark Connect-specific APIs. + """ + if isinstance(func, property): + # If it's a property, we need to set the attribute on the getter function + getter_func = func.fget + getter_func._remote_only = True # type: ignore[union-attr] + return property(getter_func) + else: + func._remote_only = True # type: ignore[attr-defined] + return func diff --git a/python/pyspark/util.py b/python/pyspark/util.py index 86779de49a2ee..3b38b8b72c615 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -382,19 +382,24 @@ def inheritable_thread_target(f: Optional[Union[Callable, "SparkSession"]] = Non assert session is not None, "Spark Connect session must be provided." def outer(ff: Callable) -> Callable: + thread_local = session.client.thread_local # type: ignore[union-attr, operator] session_client_thread_local_attrs = [ (attr, copy.deepcopy(value)) for ( attr, value, - ) in session.client.thread_local.__dict__.items() # type: ignore[union-attr] + ) in thread_local.__dict__.items() ] @functools.wraps(ff) def inner(*args: Any, **kwargs: Any) -> Any: # Set thread locals in child thread. for attr, value in session_client_thread_local_attrs: - setattr(session.client.thread_local, attr, value) # type: ignore[union-attr] + setattr( + session.client.thread_local, # type: ignore[union-attr, operator] + attr, + value, + ) return ff(*args, **kwargs) return inner @@ -489,7 +494,8 @@ def __init__( def copy_local_properties(*a: Any, **k: Any) -> Any: # Set tags in child thread. assert hasattr(self, "_tags") - session.client.thread_local.tags = self._tags # type: ignore[union-attr, has-type] + thread_local = session.client.thread_local # type: ignore[union-attr, operator] + thread_local.tags = self._tags # type: ignore[has-type] return target(*a, **k) super(InheritableThread, self).__init__( @@ -523,9 +529,10 @@ def start(self) -> None: if is_remote(): # Spark Connect assert hasattr(self, "_session") - if not hasattr(self._session.client.thread_local, "tags"): - self._session.client.thread_local.tags = set() - self._tags = set(self._session.client.thread_local.tags) + thread_local = self._session.client.thread_local # type: ignore[union-attr, operator] + if not hasattr(thread_local, "tags"): + thread_local.tags = set() + self._tags = set(thread_local.tags) else: # Non Spark Connect from pyspark import SparkContext From 0f1e410a94d3bab62c6cf0aba21ad58b40aa037c Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Thu, 21 Nov 2024 09:39:50 +0100 Subject: [PATCH 35/58] [SPARK-50016][SQL] Assign appropriate error condition for `_LEGACY_ERROR_TEMP_2067`: `UNSUPPORTED_PARTITION_TRANSFORM` ### What changes were proposed in this pull request? This PR proposes to assign proper error condition & sqlstate for `_LEGACY_ERROR_TEMP_2067`: `UNSUPPORTED_PARTITION_TRANSFORM` ### Why are the changes needed? To improve the error message by assigning proper error condition and SQLSTATE ### Does this PR introduce _any_ user-facing change? No, only user-facing error message improved ### How was this patch tested? Updated the existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48655 from itholic/LEGACY_2067. Authored-by: Haejoon Lee Signed-off-by: Max Gekk --- .../src/main/resources/error/error-conditions.json | 11 ++++++----- .../spark/sql/errors/QueryExecutionErrors.scala | 4 ++-- .../sql/execution/command/PlanResolutionSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index dfa22d5ec125e..272a0f92431ed 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5552,6 +5552,12 @@ }, "sqlState" : "42902" }, + "UNSUPPORTED_PARTITION_TRANSFORM" : { + "message" : [ + "Unsupported partition transform: . The supported transforms are `identity`, `bucket`, and `clusterBy`. Ensure your transform expression uses one of these." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_SAVE_MODE" : { "message" : [ "The save mode is not supported for:" @@ -7090,11 +7096,6 @@ "Invalid namespace name: ." ] }, - "_LEGACY_ERROR_TEMP_2067" : { - "message" : [ - "Unsupported partition transform: ." - ] - }, "_LEGACY_ERROR_TEMP_2068" : { "message" : [ "Missing database location." diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index ef4fdeeaddfe4..0852e773c87b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -872,8 +872,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def unsupportedPartitionTransformError( transform: Transform): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2067", - messageParameters = Map("transform" -> transform.toString())) + errorClass = "UNSUPPORTED_PARTITION_TRANSFORM", + messageParameters = Map("transform" -> toSQLId(transform.toString))) } def missingDatabaseLocationError(): SparkIllegalArgumentException = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index 5a4d7c86761fe..92467cbcb6c05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -306,8 +306,8 @@ class PlanResolutionSuite extends AnalysisTest { exception = intercept[SparkUnsupportedOperationException] { parseAndResolve(query) }, - condition = "_LEGACY_ERROR_TEMP_2067", - parameters = Map("transform" -> transform)) + condition = "UNSUPPORTED_PARTITION_TRANSFORM", + parameters = Map("transform" -> toSQLId(transform))) } } From b05ef451061f9ff7ba15bf037199339ed7236748 Mon Sep 17 00:00:00 2001 From: Stefan Kandic Date: Thu, 21 Nov 2024 17:38:57 +0800 Subject: [PATCH 36/58] [SPARK-50175][SQL] Change collation precedence calculation ### What changes were proposed in this pull request? Changing the way how the collation strength of string expressions are calculated. Currently, there are three different collation strengths: - explicit - result of the `collate` expression - implicit - column references and output of string functions - default - literals and cast expression However, unlike in other database systems (pg, sql server) collation strengths were not propagated up the expression tree, meaning that `substring('a' collate unicode), 0, 1)` would have implicit strength because it is the result of a string expression. My proposal is to change the behavior to be more in line with other systems mentioned above; and to do it by traversing the expression tree and propagating the highest precedence strengths up (explicit being the highest and default the lowest) while also finding the conflicts between them (conflicting explicit or implicit strenghts). ### Why are the changes needed? To be more consistent with other systems that have collations (postgre, sql server etc.) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48663 from stefankandic/newCollationPrec-separate. Authored-by: Stefan Kandic Signed-off-by: Wenchen Fan --- .../analysis/CollationTypeCoercion.scala | 347 +++++++++++------ .../expressions/complexTypeCreator.scala | 7 +- .../expressions/complexTypeExtractors.scala | 1 + .../expressions/stringExpressions.scala | 8 +- .../sql/CollationExpressionWalkerSuite.scala | 2 +- .../sql/CollationSQLExpressionsSuite.scala | 11 +- .../sql/CollationStringExpressionsSuite.scala | 6 +- .../org/apache/spark/sql/CollationSuite.scala | 64 ++-- .../CollationTypePrecedenceSuite.scala | 361 +++++++++++++++++- 9 files changed, 630 insertions(+), 177 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala index 1e9c3aabedb33..532e5e0d0a066 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CollationTypeCoercion.scala @@ -17,95 +17,51 @@ package org.apache.spark.sql.catalyst.analysis -import javax.annotation.Nullable - import scala.annotation.tailrec +import org.apache.spark.sql.catalyst.analysis.CollationStrength.{Default, Explicit, Implicit} import org.apache.spark.sql.catalyst.analysis.TypeCoercion.{hasStringType, haveSameType} -import org.apache.spark.sql.catalyst.expressions.{ - ArrayAppend, - ArrayContains, - ArrayExcept, - ArrayIntersect, - ArrayJoin, - ArrayPosition, - ArrayRemove, - ArraysOverlap, - ArrayUnion, - CaseWhen, - Cast, - Coalesce, - Collate, - Concat, - ConcatWs, - Contains, - CreateArray, - CreateMap, - Elt, - EndsWith, - EqualNullSafe, - EqualTo, - Expression, - FindInSet, - GetMapValue, - GreaterThan, - GreaterThanOrEqual, - Greatest, - If, - In, - InSubquery, - Lag, - Lead, - Least, - LessThan, - LessThanOrEqual, - Levenshtein, - Literal, - Mask, - Overlay, - RaiseError, - RegExpReplace, - SplitPart, - StartsWith, - StringInstr, - StringLocate, - StringLPad, - StringReplace, - StringRPad, - StringSplitSQL, - StringToMap, - StringTranslate, - StringTrim, - StringTrimLeft, - StringTrimRight, - SubstringIndex, - ToNumber, - TryToNumber -} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StringType} +import org.apache.spark.sql.util.SchemaUtils /** * Type coercion helper that matches against expressions in order to apply collation type coercion. */ object CollationTypeCoercion { + + private val COLLATION_CONTEXT_TAG = new TreeNodeTag[CollationContext]("collationContext") + + private def hasCollationContextTag(expr: Expression): Boolean = { + expr.getTagValue(COLLATION_CONTEXT_TAG).isDefined + } + def apply(expression: Expression): Expression = expression match { + case cast: Cast if shouldRemoveCast(cast) => + cast.child + case ifExpr: If => ifExpr.withNewChildren( ifExpr.predicate +: collateToSingleType(Seq(ifExpr.trueValue, ifExpr.falseValue)) ) case caseWhenExpr: CaseWhen if !haveSameType(caseWhenExpr.inputTypesForMerging) => - val outputStringType = - getOutputCollation(caseWhenExpr.branches.map(_._2) ++ caseWhenExpr.elseValue) - val newBranches = caseWhenExpr.branches.map { - case (condition, value) => - (condition, castStringType(value, outputStringType).getOrElse(value)) + val outputStringType = findLeastCommonStringType( + caseWhenExpr.branches.map(_._2) ++ caseWhenExpr.elseValue) + outputStringType match { + case Some(st) => + val newBranches = caseWhenExpr.branches.map { case (condition, value) => + (condition, castStringType(value, st)) + } + val newElseValue = + caseWhenExpr.elseValue.map(e => castStringType(e, st)) + CaseWhen(newBranches, newElseValue) + + case _ => + caseWhenExpr } - val newElseValue = - caseWhenExpr.elseValue.map(e => castStringType(e, outputStringType).getOrElse(e)) - CaseWhen(newBranches, newElseValue) case stringLocate: StringLocate => stringLocate.withNewChildren( @@ -156,6 +112,12 @@ object CollationTypeCoercion { val newValues = collateToSingleType(mapCreate.values) mapCreate.withNewChildren(newKeys.zip(newValues).flatMap(pair => Seq(pair._1, pair._2))) + case namedStruct: CreateNamedStruct if namedStruct.children.size % 2 == 0 => + val newNames = collateToSingleType(namedStruct.nameExprs) + val newValues = collateToSingleType(namedStruct.valExprs) + val interleaved = newNames.zip(newValues).flatMap(pair => Seq(pair._1, pair._2)) + namedStruct.withNewChildren(interleaved) + case splitPart: SplitPart => val Seq(str, delimiter, partNum) = splitPart.children val Seq(newStr, newDelimiter) = collateToSingleType(Seq(str, delimiter)) @@ -193,88 +155,221 @@ object CollationTypeCoercion { case other => other } + /** + * If childType is collated and target is UTF8_BINARY, the collation of the output + * should be that of the childType. + */ + private def shouldRemoveCast(cast: Cast): Boolean = { + val isUserDefined = cast.getTagValue(Cast.USER_SPECIFIED_CAST).isDefined + val isChildTypeCollatedString = cast.child.dataType match { + case st: StringType => !st.isUTF8BinaryCollation + case _ => false + } + val targetType = cast.dataType + + isUserDefined && isChildTypeCollatedString && targetType == StringType + } + /** * Extracts StringTypes from filtered hasStringType */ @tailrec - private def extractStringType(dt: DataType): StringType = dt match { - case st: StringType => st + private def extractStringType(dt: DataType): Option[StringType] = dt match { + case st: StringType => Some(st) case ArrayType(et, _) => extractStringType(et) + case _ => None } /** * Casts given expression to collated StringType with id equal to collationId only * if expression has StringType in the first place. - * @param expr - * @param collationId - * @return */ - def castStringType(expr: Expression, st: StringType): Option[Expression] = - castStringType(expr.dataType, st).map { dt => Cast(expr, dt)} + def castStringType(expr: Expression, st: StringType): Expression = { + castStringType(expr.dataType, st) + .map(dt => Cast(expr, dt)) + .getOrElse(expr) + } private def castStringType(inType: DataType, castType: StringType): Option[DataType] = { - @Nullable val ret: DataType = inType match { - case st: StringType if st.collationId != castType.collationId => castType + inType match { + case st: StringType if st.collationId != castType.collationId => + Some(castType) case ArrayType(arrType, nullable) => - castStringType(arrType, castType).map(ArrayType(_, nullable)).orNull - case _ => null + castStringType(arrType, castType).map(ArrayType(_, nullable)) + case _ => None } - Option(ret) } /** * Collates input expressions to a single collation. */ - def collateToSingleType(exprs: Seq[Expression]): Seq[Expression] = { - val st = getOutputCollation(exprs) + def collateToSingleType(expressions: Seq[Expression]): Seq[Expression] = { + val lctOpt = findLeastCommonStringType(expressions) - exprs.map(e => castStringType(e, st).getOrElse(e)) + lctOpt match { + case Some(lct) => + expressions.map(e => castStringType(e, lct)) + case _ => + expressions + } } /** - * Based on the data types of the input expressions this method determines - * a collation type which the output will have. This function accepts Seq of - * any expressions, but will only be affected by collated StringTypes or - * complex DataTypes with collated StringTypes (e.g. ArrayType) + * Tries to find the least common StringType among the given expressions. */ - def getOutputCollation(expr: Seq[Expression]): StringType = { - val explicitTypes = expr.filter { - case _: Collate => true - case _ => false - } - .map(_.dataType.asInstanceOf[StringType].collationId) - .distinct - - explicitTypes.size match { - // We have 1 explicit collation - case 1 => StringType(explicitTypes.head) - // Multiple explicit collations occurred - case size if size > 1 => - throw QueryCompilationErrors - .explicitCollationMismatchError( - explicitTypes.map(t => StringType(t)) - ) - // Only implicit or default collations present - case 0 => - val implicitTypes = expr.filter { - case Literal(_, _: StringType) => false - case cast: Cast if cast.getTagValue(Cast.USER_SPECIFIED_CAST).isEmpty => - cast.child.dataType.isInstanceOf[StringType] - case _ => true - } - .map(_.dataType) - .filter(hasStringType) - .map(extractStringType(_).collationId) - .distinct - - if (implicitTypes.length > 1) { - throw QueryCompilationErrors.implicitCollationMismatchError( - implicitTypes.map(t => StringType(t)) - ) + private def findLeastCommonStringType(expressions: Seq[Expression]): Option[StringType] = { + if (!expressions.exists(e => SchemaUtils.hasNonUTF8BinaryCollation(e.dataType))) { + return None + } + + val collationContextWinner = expressions.foldLeft(findCollationContext(expressions.head)) { + case (Some(left), right) => + findCollationContext(right).flatMap { ctx => + collationPrecedenceWinner(left, ctx) } - else { - implicitTypes.headOption.map(StringType(_)).getOrElse(SQLConf.get.defaultStringType) + case (None, _) => return None + } + + collationContextWinner.flatMap { cc => + extractStringType(cc.dataType) + } + } + + /** + * Tries to find the collation context for the given expression. + * If found, it will also set the [[COLLATION_CONTEXT_TAG]] on the expression, + * so that the context can be reused later. + */ + private def findCollationContext(expr: Expression): Option[CollationContext] = { + val contextOpt = expr match { + case _ if hasCollationContextTag(expr) => + Some(expr.getTagValue(COLLATION_CONTEXT_TAG).get) + + // if `expr` doesn't have a string in its dataType then it doesn't + // have the collation context either + case _ if !expr.dataType.existsRecursively(_.isInstanceOf[StringType]) => + None + + case collate: Collate => + Some(CollationContext(collate.dataType, Explicit)) + + case _: Alias | _: SubqueryExpression | _: AttributeReference | _: VariableReference => + Some(CollationContext(expr.dataType, Implicit)) + + case _: Literal => + Some(CollationContext(expr.dataType, Default)) + + // if it does have a string type but none of its children do + // then the collation context strength is default + case _ if !expr.children.exists(_.dataType.existsRecursively(_.isInstanceOf[StringType])) => + Some(CollationContext(expr.dataType, Default)) + + case _ => + val contextWinnerOpt = getContextRelevantChildren(expr) + .flatMap(findCollationContext) + .foldLeft(Option.empty[CollationContext]) { + case (Some(left), right) => + collationPrecedenceWinner(left, right) + case (None, right) => + Some(right) + } + + contextWinnerOpt.map { context => + if (hasStringType(expr.dataType)) { + CollationContext(expr.dataType, context.strength) + } else { + context + } } } + + contextOpt.foreach(expr.setTagValue(COLLATION_CONTEXT_TAG, _)) + contextOpt + } + + /** + * Returns the children of the given expression that should be used for calculating the + * winning collation context. + */ + private def getContextRelevantChildren(expression: Expression): Seq[Expression] = { + expression match { + // collation context for named struct should be calculated based on its values only + case createStruct: CreateNamedStruct => + createStruct.valExprs + + // collation context does not depend on the key for extracting the value + case extract: ExtractValue => + Seq(extract.child) + + // we currently don't support collation precedence for maps, + // as this would involve calculating them for keys and values separately + case _: CreateMap => + Seq.empty + + case _ => + expression.children + } + } + + /** + * Returns the collation context that wins in precedence between left and right. + */ + private def collationPrecedenceWinner( + left: CollationContext, + right: CollationContext): Option[CollationContext] = { + + val (leftStringType, rightStringType) = + (extractStringType(left.dataType), extractStringType(right.dataType)) match { + case (Some(l), Some(r)) => + (l, r) + case (None, None) => + return None + case (Some(_), None) => + return Some(left) + case (None, Some(_)) => + return Some(right) + } + + (left.strength, right.strength) match { + case (Explicit, Explicit) if leftStringType != rightStringType => + throw QueryCompilationErrors.explicitCollationMismatchError( + Seq(leftStringType, rightStringType)) + + case (Explicit, _) => Some(left) + case (_, Explicit) => Some(right) + + case (Implicit, Implicit) if leftStringType != rightStringType => + throw QueryCompilationErrors.implicitCollationMismatchError( + Seq(leftStringType, rightStringType)) + + case (Implicit, _) => Some(left) + case (_, Implicit) => Some(right) + + case (Default, Default) if leftStringType != rightStringType => + throw QueryCompilationErrors.implicitCollationMismatchError( + Seq(leftStringType, rightStringType)) + + case _ => + Some(left) + } } } + +/** + * Represents the strength of collation used for determining precedence in collation resolution. + */ +private sealed trait CollationStrength {} + + private object CollationStrength { + case object Explicit extends CollationStrength {} + case object Implicit extends CollationStrength {} + case object Default extends CollationStrength {} +} + +/** + * Encapsulates the context for collation, including data type and strength. + * + * @param dataType The data type associated with this collation context. + * @param strength The strength level of the collation, which determines its precedence. + */ +private case class CollationContext(dataType: DataType, strength: CollationStrength) {} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 2098ee274dfe0..e7cc174f7cf39 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -565,11 +565,14 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E extends TernaryExpression with ExpectsInputTypes { override def nullIntolerant: Boolean = true def this(child: Expression, pairDelim: Expression) = { - this(child, pairDelim, Literal(":")) + this(child, pairDelim, Literal.create(":", SQLConf.get.defaultStringType)) } def this(child: Expression) = { - this(child, Literal(","), Literal(":")) + this( + child, + Literal.create(",", SQLConf.get.defaultStringType), + Literal.create(":", SQLConf.get.defaultStringType)) } override def stateful: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 3b8d4e09905e5..2013cd8d6e636 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -93,6 +93,7 @@ object ExtractValue { trait ExtractValue extends Expression { override def nullIntolerant: Boolean = true final override val nodePatterns: Seq[TreePattern] = Seq(EXTRACT_VALUE) + val child: Expression } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index d92f45b1968ab..c97920619ba4d 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -1862,7 +1862,8 @@ trait PadExpressionBuilderBase extends ExpressionBuilder { if (expressions(0).dataType == BinaryType && behaviorChangeEnabled) { BinaryPad(funcName, expressions(0), expressions(1), Literal(Array[Byte](0))) } else { - createStringPad(expressions(0), expressions(1), Literal(" ")) + createStringPad(expressions(0), + expressions(1), Literal.create(" ", SQLConf.get.defaultStringType)) } } else if (numArgs == 3) { if (expressions(0).dataType == BinaryType && expressions(2).dataType == BinaryType @@ -1992,7 +1993,10 @@ object RPadExpressionBuilder extends PadExpressionBuilderBase { } } -case class StringRPad(str: Expression, len: Expression, pad: Expression = Literal(" ")) +case class StringRPad( + str: Expression, + len: Expression, + pad: Expression = Literal.create(" ", SQLConf.get.defaultStringType)) extends TernaryExpression with ImplicitCastInputTypes { override def nullIntolerant: Boolean = true override def first: Expression = str diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index 2b49b76ff8c7a..bc62fa5fdd331 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -742,7 +742,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi } } catch { case e: SparkRuntimeException => assert(e.getCondition == "USER_RAISED_EXCEPTION") - case other: Throwable => throw other + case other: Throwable => throw new Exception(s"Query $query failed", other) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index 3563e04dced11..6feb4587b816f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -2416,8 +2416,15 @@ class CollationSQLExpressionsSuite |collate('${testCase.left}', '${testCase.leftCollation}'))= |collate('${testCase.right}', '${testCase.rightCollation}'); |""".stripMargin - val testQuery = sql(query) - checkAnswer(testQuery, Row(testCase.result)) + + if (testCase.leftCollation == testCase.rightCollation) { + checkAnswer(sql(query), Row(testCase.result)) + } else { + val exception = intercept[AnalysisException] { + sql(query) + } + assert(exception.getCondition === "COLLATION_MISMATCH.EXPLICIT") + } }) val queryPass = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 9ee2cfb964feb..2a0b84c075079 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -180,11 +180,12 @@ class CollationStringExpressionsSuite // Because `StringSplitSQL` is an internal expression, // E2E SQL test cannot be performed in `collations.sql`. + checkError( exception = intercept[AnalysisException] { val expr = StringSplitSQL( - Cast(Literal.create("1a2"), StringType("UTF8_BINARY")), - Cast(Literal.create("a"), StringType("UTF8_LCASE"))) + Literal.create("1a2", StringType("UTF8_BINARY")), + Literal.create("a", StringType("UTF8_LCASE"))) CollationTypeCasts.transform(expr) }, condition = "COLLATION_MISMATCH.IMPLICIT", @@ -193,6 +194,7 @@ class CollationStringExpressionsSuite "implicitTypes" -> """"STRING", "STRING COLLATE UTF8_LCASE"""" ) ) + checkError( exception = intercept[AnalysisException] { val expr = StringSplitSQL( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index f5cb30809ae50..1707820053837 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -624,17 +624,11 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { checkAnswer(sql(s"SELECT c1 FROM $tableName WHERE c1 = COLLATE('a', 'UTF8_BINARY')"), Seq(Row("a"))) - // fail with implicit mismatch, as function return should be considered implicit - checkError( - exception = intercept[AnalysisException] { - sql(s"SELECT c1 FROM $tableName " + - s"WHERE c1 = SUBSTR(COLLATE('a', 'UNICODE'), 0)") - }, - condition = "COLLATION_MISMATCH.IMPLICIT", - parameters = Map( - "implicitTypes" -> """"STRING COLLATE UTF8_LCASE", "STRING COLLATE UNICODE"""" - ) - ) + // explicit collation propagates up + checkAnswer( + sql(s"SELECT c1 FROM $tableName " + + s"WHERE c1 = SUBSTR(COLLATE('a', 'UNICODE'), 0)"), + Row("a")) // in operator checkAnswer(sql(s"SELECT c1 FROM $tableName WHERE c1 IN ('a')"), @@ -742,9 +736,16 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // concat + in - checkAnswer(sql(s"SELECT c1 FROM $tableName WHERE c1 || COLLATE('a', 'UTF8_BINARY') IN " + - s"(COLLATE('aa', 'UNICODE'))"), - Seq(Row("a"))) + checkError( + exception = intercept[AnalysisException] { + sql(s"SELECT c1 FROM $tableName where c1 || COLLATE('a', 'UTF8_BINARY') IN " + + s"(COLLATE('aa', 'UNICODE'))") + }, + condition = "COLLATION_MISMATCH.EXPLICIT", + parameters = Map( + "explicitTypes" -> """"STRING", "STRING COLLATE UNICODE"""" + ) + ) // array creation supports implicit casting checkAnswer(sql(s"SELECT typeof(array('a' COLLATE UNICODE, 'b')[1])"), @@ -765,14 +766,21 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException] { sql(s"SELECT array('A', 'a' COLLATE UNICODE) == array('b' COLLATE UNICODE_CI)") }, - condition = "COLLATION_MISMATCH.IMPLICIT", + condition = "COLLATION_MISMATCH.EXPLICIT", parameters = Map( - "implicitTypes" -> """"STRING COLLATE UNICODE", "STRING COLLATE UNICODE_CI"""" + "explicitTypes" -> """"STRING COLLATE UNICODE", "STRING COLLATE UNICODE_CI"""" ) ) - checkAnswer(sql("SELECT array_join(array('a', 'b' collate UNICODE), 'c' collate UNICODE_CI)"), - Seq(Row("acb"))) + checkError( + exception = intercept[AnalysisException] { + sql("SELECT array_join(array('a', 'b' collate UNICODE), 'c' collate UNICODE_CI)") + }, + condition = "COLLATION_MISMATCH.EXPLICIT", + parameters = Map( + "explicitTypes" -> """"STRING COLLATE UNICODE", "STRING COLLATE UNICODE_CI"""" + ) + ) } } @@ -851,26 +859,6 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } - test("SPARK-47692: Parameter markers with variable mapping") { - checkAnswer( - spark.sql( - "SELECT collation(:var1 || :var2)", - Map("var1" -> Literal.create('a', StringType("UTF8_BINARY")), - "var2" -> Literal.create('b', StringType("UNICODE")))), - Seq(Row("UTF8_BINARY")) - ) - - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UNICODE") { - checkAnswer( - spark.sql( - "SELECT collation(:var1 || :var2)", - Map("var1" -> Literal.create('a', StringType("UTF8_BINARY")), - "var2" -> Literal.create('b', StringType("UNICODE")))), - Seq(Row("UNICODE")) - ) - } - } - test("SPARK-47210: Cast of default collated strings in IN expression") { val tableName = "t1" withTable(tableName) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala index 6f10acf264b07..4a904a85e0a7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationTypePrecedenceSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.collation import org.apache.spark.SparkThrowable -import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.connector.DatasourceV2SQLBase -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession -class CollationTypePrecedenceSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { +class CollationTypePrecedenceSuite extends QueryTest with SharedSparkSession { val dataSource: String = "parquet" @@ -33,6 +32,360 @@ class CollationTypePrecedenceSuite extends DatasourceV2SQLBase with AdaptiveSpar assert(exception.getCondition === errorClass) } + private def assertExplicitMismatch(df: => DataFrame): Unit = + assertThrowsError(df, "COLLATION_MISMATCH.EXPLICIT") + + private def assertImplicitMismatch(df: => DataFrame): Unit = + assertThrowsError(df, "COLLATION_MISMATCH.IMPLICIT") + + test("explicit collation propagates up") { + checkAnswer( + sql(s"SELECT COLLATION('a' collate unicode)"), + Row("UNICODE")) + + checkAnswer( + sql(s"SELECT COLLATION('a' collate unicode || 'b')"), + Row("UNICODE")) + + checkAnswer( + sql(s"SELECT COLLATION(SUBSTRING('a' collate unicode, 0, 1))"), + Row("UNICODE")) + + checkAnswer( + sql(s"SELECT COLLATION(SUBSTRING('a' collate unicode, 0, 1) || 'b')"), + Row("UNICODE")) + + assertExplicitMismatch( + sql(s"SELECT COLLATION('a' collate unicode || 'b' collate utf8_lcase)")) + + assertExplicitMismatch( + sql(s""" + |SELECT COLLATION( + | SUBSTRING('a' collate unicode, 0, 1) || + | SUBSTRING('b' collate utf8_lcase, 0, 1)) + |""".stripMargin)) + } + + test("implicit collation in columns") { + val tableName = "implicit_coll_tbl" + val c1Collation = "UNICODE" + val c2Collation = "UNICODE_CI" + val structCollation = "UTF8_LCASE" + withTable(tableName) { + sql(s""" + |CREATE TABLE $tableName ( + | c1 STRING COLLATE $c1Collation, + | c2 STRING COLLATE $c2Collation, + | c3 STRUCT) + |""".stripMargin) + sql(s"INSERT INTO $tableName VALUES ('a', 'b', named_struct('col1', 'c'))") + + checkAnswer( + sql(s"SELECT COLLATION(c1 || 'a') FROM $tableName"), + Seq(Row(c1Collation))) + + checkAnswer( + sql(s"SELECT COLLATION(c3.col1 || 'a') FROM $tableName"), + Seq(Row(structCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(SUBSTRING(c1, 0, 1) || 'a') FROM $tableName"), + Seq(Row(c1Collation))) + + assertImplicitMismatch(sql(s"SELECT COLLATION(c1 || c2) FROM $tableName")) + assertImplicitMismatch(sql(s"SELECT COLLATION(c1 || c3.col1) FROM $tableName")) + assertImplicitMismatch( + sql(s"SELECT COLLATION(SUBSTRING(c1, 0, 1) || c2) FROM $tableName")) + } + } + + test("variables have implicit collation") { + val v1Collation = "UTF8_BINARY" + val v2Collation = "UTF8_LCASE" + sql(s"DECLARE v1 = 'a'") + sql(s"DECLARE v2 = 'b' collate $v2Collation") + + checkAnswer( + sql(s"SELECT COLLATION(v1 || 'a')"), + Row(v1Collation)) + + checkAnswer( + sql(s"SELECT COLLATION(v2 || 'a')"), + Row(v2Collation)) + + checkAnswer( + sql(s"SELECT COLLATION(v2 || 'a' COLLATE UTF8_BINARY)"), + Row("UTF8_BINARY")) + + checkAnswer( + sql(s"SELECT COLLATION(SUBSTRING(v2, 0, 1) || 'a')"), + Row(v2Collation)) + + assertImplicitMismatch(sql(s"SELECT COLLATION(v1 || v2)")) + assertImplicitMismatch(sql(s"SELECT COLLATION(SUBSTRING(v1, 0, 1) || v2)")) + } + + test("subqueries have implicit collation strength") { + withTable("t") { + sql(s"CREATE TABLE t (c STRING COLLATE UTF8_LCASE) USING $dataSource") + + sql(s"SELECT (SELECT 'text' COLLATE UTF8_BINARY) || c collate UTF8_BINARY from t") + assertImplicitMismatch( + sql(s"SELECT (SELECT 'text' COLLATE UTF8_BINARY) || c from t")) + } + + // Simple subquery with explicit collation + checkAnswer( + sql(s"SELECT COLLATION((SELECT 'text' COLLATE UTF8_BINARY) || 'suffix')"), + Row("UTF8_BINARY") + ) + + checkAnswer( + sql(s"SELECT COLLATION((SELECT 'text' COLLATE UTF8_LCASE) || 'suffix')"), + Row("UTF8_LCASE") + ) + + // Nested subquery should retain the collation of the deepest expression + checkAnswer( + sql(s"SELECT COLLATION((SELECT (SELECT 'inner' COLLATE UTF8_LCASE) || 'outer'))"), + Row("UTF8_LCASE") + ) + + checkAnswer( + sql(s"SELECT COLLATION((SELECT (SELECT 'inner' COLLATE UTF8_BINARY) || 'outer'))"), + Row("UTF8_BINARY") + ) + + // Subqueries with mixed collations should follow collation precedence rules + checkAnswer( + sql(s"SELECT COLLATION((SELECT 'string1' COLLATE UTF8_LCASE || " + + s"(SELECT 'string2' COLLATE UTF8_BINARY)))"), + Row("UTF8_LCASE") + ) + } + + test("struct test") { + val tableName = "struct_tbl" + val c1Collation = "UNICODE_CI" + val c2Collation = "UNICODE" + withTable(tableName) { + sql(s""" + |CREATE TABLE $tableName ( + | c1 STRUCT, + | c2 STRUCT>) + |USING $dataSource + |""".stripMargin) + sql(s"INSERT INTO $tableName VALUES (named_struct('col1', 'a')," + + s"named_struct('col1', named_struct('col1', 'c')))") + + checkAnswer( + sql(s"SELECT COLLATION(c2.col1.col1 || 'a') FROM $tableName"), + Seq(Row(c2Collation))) + + checkAnswer( + sql(s"SELECT COLLATION(c1.col1 || 'a') FROM $tableName"), + Seq(Row(c1Collation))) + + checkAnswer( + sql(s"SELECT COLLATION(c1.col1 || 'a' collate UNICODE) FROM $tableName"), + Seq(Row("UNICODE"))) + + checkAnswer( + sql(s"SELECT COLLATION(struct('a').col1 || 'a' collate UNICODE) FROM $tableName"), + Seq(Row("UNICODE"))) + + checkAnswer( + sql(s"SELECT COLLATION(struct('a' collate UNICODE).col1 || 'a') FROM $tableName"), + Seq(Row("UNICODE"))) + + checkAnswer( + sql(s"SELECT COLLATION(struct('a').col1 collate UNICODE || 'a' collate UNICODE) " + + s"FROM $tableName"), + Seq(Row("UNICODE"))) + + assertExplicitMismatch( + sql(s"SELECT COLLATION(struct('a').col1 collate UNICODE || 'a' collate UTF8_LCASE) " + + s"FROM $tableName")) + + assertExplicitMismatch( + sql(s"SELECT COLLATION(struct('a' collate UNICODE).col1 || 'a' collate UTF8_LCASE) " + + s"FROM $tableName")) + } + } + + test("array test") { + val tableName = "array_tbl" + val columnCollation = "UNICODE" + val arrayCollation = "UNICODE_CI" + withTable(tableName) { + sql(s""" + |CREATE TABLE $tableName ( + | c1 STRING COLLATE $columnCollation, + | c2 ARRAY) + |USING $dataSource + |""".stripMargin) + + sql(s"INSERT INTO $tableName VALUES ('a', array('b', 'c'))") + + checkAnswer( + sql(s"SELECT collation(element_at(array('a', 'b' collate utf8_lcase), 1))"), + Seq(Row("UTF8_LCASE"))) + + assertExplicitMismatch( + sql(s"SELECT collation(element_at(array('a' collate unicode, 'b' collate utf8_lcase), 1))") + ) + + checkAnswer( + sql(s"SELECT collation(element_at(array('a', 'b' collate utf8_lcase), 1) || c1)" + + s"from $tableName"), + Seq(Row("UTF8_LCASE"))) + + checkAnswer( + sql(s"SELECT collation(element_at(array_append(c2, 'd'), 1)) FROM $tableName"), + Seq(Row(arrayCollation)) + ) + + checkAnswer( + sql(s"SELECT collation(element_at(array_append(c2, 'd' collate utf8_lcase), 1))" + + s"FROM $tableName"), + Seq(Row("UTF8_LCASE")) + ) + } + } + + test("array cast") { + val tableName = "array_cast_tbl" + val columnCollation = "UNICODE" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (c1 ARRAY) USING $dataSource") + sql(s"INSERT INTO $tableName VALUES (array('a'))") + + checkAnswer( + sql(s"SELECT COLLATION(c1[0]) FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(cast(c1 AS ARRAY)[0]) FROM $tableName"), + Seq(Row("UTF8_BINARY"))) + + checkAnswer( + sql(s"SELECT COLLATION(cast(c1 AS ARRAY)[0]) FROM $tableName"), + Seq(Row("UTF8_LCASE"))) + } + } + + test("user defined cast") { + val tableName = "dflt_coll_tbl" + val columnCollation = "UNICODE" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (c1 STRING COLLATE $columnCollation) USING $dataSource") + sql(s"INSERT INTO $tableName VALUES ('a')") + + // only for non string inputs cast results in default collation + checkAnswer( + sql(s"SELECT COLLATION(c1 || CAST(to_char(DATE'2016-04-08', 'y') AS STRING)) " + + s"FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(CAST(to_char(DATE'2016-04-08', 'y') AS STRING)) " + + s"FROM $tableName"), + Seq(Row("UTF8_BINARY"))) + + // for string inputs collation is of the child expression + checkAnswer( + sql(s"SELECT COLLATION(CAST('a' AS STRING)) FROM $tableName"), + Seq(Row("UTF8_BINARY"))) + + checkAnswer( + sql(s"SELECT COLLATION(CAST(c1 AS STRING)) FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(CAST(c1 collate UTF8_LCASE AS STRING)) FROM $tableName"), + Seq(Row("UTF8_LCASE"))) + + checkAnswer( + sql(s"SELECT COLLATION(c1 || CAST('a' AS STRING)) FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(c1 || CAST('a' collate UTF8_LCASE AS STRING)) FROM $tableName"), + Seq(Row("UTF8_LCASE"))) + + checkAnswer( + sql(s"SELECT COLLATION(c1 || CAST(c1 AS STRING)) FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION(c1 || SUBSTRING(CAST(c1 AS STRING), 0, 1)) FROM $tableName"), + Seq(Row(columnCollation))) + } + } + + test("str fns without params have default strength") { + val tableName = "str_fns_tbl" + val columnCollation = "UNICODE" + withTable(tableName) { + sql(s"CREATE TABLE $tableName (c1 STRING COLLATE $columnCollation) USING $dataSource") + sql(s"INSERT INTO $tableName VALUES ('a')") + + checkAnswer( + sql(s"SELECT COLLATION('a' collate utf8_lcase || current_database()) FROM $tableName"), + Seq(Row("UTF8_LCASE"))) + + checkAnswer( + sql(s"SELECT COLLATION(c1 || current_database()) FROM $tableName"), + Seq(Row(columnCollation))) + + checkAnswer( + sql(s"SELECT COLLATION('a' || current_database()) FROM $tableName"), + Seq(Row("UTF8_BINARY"))) + } + } + + test("functions that contain both string and non string params") { + checkAnswer( + sql(s"SELECT COLLATION(elt(2, 'a', 'b'))"), + Row("UTF8_BINARY")) + + checkAnswer( + sql(s"SELECT COLLATION(elt(2, 'a' collate UTF8_LCASE, 'b'))"), + Row("UTF8_LCASE")) + + assertExplicitMismatch( + sql(s"SELECT COLLATION(elt(2, 'a' collate UTF8_LCASE, 'b' collate UNICODE))")) + } + + test("named_struct names and values") { + checkAnswer( + sql(s"SELECT named_struct('name1', 'value1', 'name2', 'value2')"), + Row(Row("value1", "value2"))) + + checkAnswer( + sql(s"SELECT named_struct" + + s"('name1' collate unicode, 'value1', 'name2' collate unicode, 'value2')"), + Row(Row("value1", "value2"))) + + checkAnswer( + sql(s"SELECT named_struct" + + s"('name1', 'value1' collate unicode, 'name2', 'value2' collate unicode)"), + Row(Row("value1", "value2"))) + + checkAnswer( + sql(s"SELECT named_struct('name1' collate utf8_lcase, 'value1' collate unicode," + + s"'name2' collate utf8_lcase, 'value2' collate unicode)"), + Row(Row("value1", "value2"))) + + assertExplicitMismatch( + sql(s"SELECT named_struct" + + s"('name1' collate unicode, 'value1', 'name2' collate utf8_lcase, 'value2')")) + + assertExplicitMismatch( + sql(s"SELECT named_struct" + + s"('name1', 'value1' collate unicode, 'name2', 'value2' collate utf8_lcase)")) + } + test("access collated map via literal") { val tableName = "map_with_lit" From fbf255ef7b01e7016f368b6afbc4234b79130af4 Mon Sep 17 00:00:00 2001 From: Mihailo Milosevic Date: Thu, 21 Nov 2024 10:55:07 +0100 Subject: [PATCH 37/58] [SPARK-50379][SQL] Fix DayTimeIntevalType handling in WindowExecBase ### What changes were proposed in this pull request? Initial PR missed adding DayTimeIntervalType handling with DateType https://github.com/apache/spark/pull/32294. This PR adds cases for DayTimeIntervalType. ### Why are the changes needed? This needs to be supported by documentation. ### Does this PR introduce _any_ user-facing change? Yes, we enable queries. ### How was this patch tested? Golden file test added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48893 from mihailom-db/SPARK-35110. Authored-by: Mihailo Milosevic Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 2 +- .../expressions/windowExpressions.scala | 2 + .../window/WindowEvaluatorFactoryBase.scala | 5 +- .../sql-tests/analyzer-results/window.sql.out | 60 +++++++++++++++++ .../resources/sql-tests/inputs/window.sql | 4 ++ .../sql-tests/results/window.sql.out | 65 +++++++++++++++++++ 6 files changed, 136 insertions(+), 2 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 272a0f92431ed..38b1656ac05c1 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -965,7 +965,7 @@ }, "RANGE_FRAME_INVALID_TYPE" : { "message" : [ - "The data type used in the order specification does not match the data type which is used in the range frame." + "The data type used in the order specification does not support the data type which is used in the range frame." ] }, "RANGE_FRAME_MULTI_ORDER" : { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index ecc32bc8d0ef9..ab787663c9923 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.trees.{BinaryLike, LeafLike, TernaryLike, U import org.apache.spark.sql.catalyst.trees.TreePattern.{TreePattern, UNRESOLVED_WINDOW_EXPRESSION, WINDOW_EXPRESSION} import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors, QueryErrorsBase, QueryExecutionErrors} import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.DayTimeIntervalType.DAY /** * The trait of the Window Specification (specified in the OVER clause or WINDOW clause) for @@ -108,6 +109,7 @@ case class WindowSpecDefinition( private def isValidFrameType(ft: DataType): Boolean = (orderSpec.head.dataType, ft) match { case (DateType, IntegerType) => true case (DateType, _: YearMonthIntervalType) => true + case (DateType, DayTimeIntervalType(DAY, DAY)) => true case (TimestampType | TimestampNTZType, CalendarIntervalType) => true case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) => true case (TimestampType | TimestampNTZType, _: DayTimeIntervalType) => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala index bdaccd43c1b7d..7d13dbbe2a06a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowEvaluatorFactoryBase.scala @@ -22,11 +22,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, WindowExpression} +import org.apache.spark.sql.catalyst.expressions.{Add, AggregateWindowFunction, Ascending, Attribute, BoundReference, CurrentRow, DateAdd, DateAddYMInterval, DecimalAddNoOverflowCheck, Descending, Expression, ExtractANSIIntervalDays, FrameLessOffsetWindowFunction, FrameType, IdentityProjection, IntegerLiteral, MutableProjection, NamedExpression, OffsetWindowFunction, PythonFuncExpression, RangeFrame, RowFrame, RowOrdering, SortOrder, SpecifiedWindowFrame, TimeAdd, TimestampAddYMInterval, UnaryMinus, UnboundedFollowing, UnboundedPreceding, UnsafeProjection, WindowExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{CalendarIntervalType, DateType, DayTimeIntervalType, DecimalType, IntegerType, TimestampNTZType, TimestampType, YearMonthIntervalType} +import org.apache.spark.sql.types.DayTimeIntervalType.DAY import org.apache.spark.util.collection.Utils trait WindowEvaluatorFactoryBase { @@ -101,6 +102,8 @@ trait WindowEvaluatorFactoryBase { val boundExpr = (expr.dataType, boundOffset.dataType) match { case (DateType, IntegerType) => DateAdd(expr, boundOffset) case (DateType, _: YearMonthIntervalType) => DateAddYMInterval(expr, boundOffset) + case (DateType, DayTimeIntervalType(DAY, DAY)) => + DateAdd(expr, ExtractANSIIntervalDays(boundOffset)) case (TimestampType | TimestampNTZType, CalendarIntervalType) => TimeAdd(expr, boundOffset, Some(timeZone)) case (TimestampType | TimestampNTZType, _: YearMonthIntervalType) => diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out index 367d5b0167014..db49b1bfd39d8 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -1414,3 +1414,63 @@ Project [cate#x, val#x, r#x] +- Project [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] +- SubqueryAlias testData +- LocalRelation [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '5' DAY PRECEDING) AS mean FROM testData +-- !query analysis +Project [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x, mean#x] ++- Project [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x, mean#x, mean#x] + +- Window [mean(val_double#x) windowspecdefinition(val#x, val_date#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, -INTERVAL '5' DAY, currentrow$())) AS mean#x], [val#x], [val_date#x ASC NULLS FIRST] + +- Project [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] + +- SubqueryAlias testdata + +- View (`testData`, [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x]) + +- Project [cast(val#x as int) AS val#x, cast(val_long#xL as bigint) AS val_long#xL, cast(val_double#x as double) AS val_double#x, cast(val_date#x as date) AS val_date#x, cast(val_timestamp#x as timestamp) AS val_timestamp#x, cast(cate#x as string) AS cate#x] + +- Project [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] + +- SubqueryAlias testData + +- LocalRelation [val#x, val_long#xL, val_double#x, val_date#x, val_timestamp#x, cate#x] + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING) AS mean FROM testData +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY val ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN INTERVAL '1 02:03:04.001' DAY TO SECOND PRECEDING AND CURRENT ROW)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 121, + "fragment" : "(partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING)" + } ] +} + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING) AS mean FROM testData +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "exprType" : "\"DATE\"", + "location" : "lower", + "sqlExpr" : "\"RANGE BETWEEN DATE '2024-01-01' FOLLOWING AND CURRENT ROW\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 102, + "fragment" : "(partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING)" + } ] +} diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index f94ff0f0a68a5..f3cbf6ef1ccb6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -478,3 +478,7 @@ SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY va SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; + +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '5' DAY PRECEDING) AS mean FROM testData; +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING) AS mean FROM testData; +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING) AS mean FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 182a4b819fcb0..87381b64638b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1442,3 +1442,68 @@ a 1 2 a NULL 1 b 1 1 b 2 2 + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '5' DAY PRECEDING) AS mean FROM testData +-- !query schema +struct +-- !query output +1 1 1.0 2017-08-01 2017-07-31 17:00:00 a 1.0 +1 2 2.5 2017-08-02 2017-08-05 23:13:20 a 1.5 +1 NULL 1.0 2017-08-01 2017-07-31 17:00:00 b 1.0 +2 2147483650 100.001 2020-12-31 2020-12-30 16:00:00 a 100.001 +2 3 3.3 2017-08-03 2017-08-17 13:00:00 b 3.3 +3 1 1.0 2017-08-01 2017-07-31 17:00:00 NULL 1.0 +3 2147483650 100.001 2020-12-31 2020-12-30 16:00:00 b 100.001 +NULL 1 1.0 2017-08-01 2017-07-31 17:00:00 a 1.0 +NULL NULL NULL NULL NULL NULL NULL + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING) AS mean FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY val ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN INTERVAL '1 02:03:04.001' DAY TO SECOND PRECEDING AND CURRENT ROW)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 121, + "fragment" : "(partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING)" + } ] +} + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING) AS mean FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "exprType" : "\"DATE\"", + "location" : "lower", + "sqlExpr" : "\"RANGE BETWEEN DATE '2024-01-01' FOLLOWING AND CURRENT ROW\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 102, + "fragment" : "(partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING)" + } ] +} From cbb16b9633afdcced2b00651d910ed2b59f09737 Mon Sep 17 00:00:00 2001 From: camilesing Date: Thu, 21 Nov 2024 11:24:13 +0100 Subject: [PATCH 38/58] [MINOR][DOCS] Fix miss semicolon on create table example sql ### What changes were proposed in this pull request? fix miss semicolon on create table example sql ### Why are the changes needed? fix miss semicolon on create table example sql. ### Does this PR introduce _any_ user-facing change? Yes. the patch fix docs miss semicolon sql. ### How was this patch tested? Manually by inspecting generated docs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48916 from camilesing/fix_docs_miss_semicolon. Authored-by: camilesing Signed-off-by: Max Gekk --- docs/sql-ref-syntax-ddl-create-table-hiveformat.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index b2f5957416a80..583e890f8305e 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -178,7 +178,7 @@ CREATE EXTERNAL TABLE family (id INT, name STRING) CREATE TABLE clustered_by_test1 (ID INT, AGE STRING) CLUSTERED BY (ID) INTO 4 BUCKETS - STORED AS ORC + STORED AS ORC; --Use `CLUSTERED BY` clause to create bucket table with `SORTED BY` CREATE TABLE clustered_by_test2 (ID INT, NAME STRING) @@ -186,7 +186,7 @@ CREATE TABLE clustered_by_test2 (ID INT, NAME STRING) CLUSTERED BY (ID, NAME) SORTED BY (ID ASC) INTO 3 BUCKETS - STORED AS PARQUET + STORED AS PARQUET; ``` ### Related Statements From f2de888fc8a5371d70a57a414795623dab75ecfd Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 21 Nov 2024 11:38:13 +0100 Subject: [PATCH 39/58] [MINOR][DOCS] Remove wrong and ambiguous default statement in datetime rebase mode settings ### What changes were proposed in this pull request? The current default values for these have been changed from EXCEPTION to CORRECTED ### Why are the changes needed? docfix ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? passing existing ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48919 from yaooqinn/minor. Authored-by: Kent Yao Signed-off-by: Max Gekk --- .../org/apache/spark/sql/internal/SQLConf.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) 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 0123f91393e7e..123759c6c8b80 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 @@ -4592,7 +4592,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase INT96 timestamps from Proleptic Gregorian calendar to " + "the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " + "When CORRECTED, Spark will not do rebase and write the timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the writing if it sees ancient " + + "When EXCEPTION, Spark will fail the writing if it sees ancient " + "timestamps that are ambiguous between the two calendars.") .version("3.1.0") .stringConf @@ -4606,7 +4606,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " + "to the legacy hybrid (Julian + Gregorian) calendar when writing Parquet files. " + "When CORRECTED, Spark will not do rebase and write the dates/timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the writing if it sees " + + "When EXCEPTION, Spark will fail the writing if it sees " + "ancient dates/timestamps that are ambiguous between the two calendars. " + "This config influences on writes of the following parquet logical types: DATE, " + "TIMESTAMP_MILLIS, TIMESTAMP_MICROS. The INT96 type has the separate config: " + @@ -4623,7 +4623,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase INT96 timestamps from the legacy hybrid (Julian + " + "Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " + "When CORRECTED, Spark will not do rebase and read the timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the reading if it sees ancient " + + "When EXCEPTION, Spark will fail the reading if it sees ancient " + "timestamps that are ambiguous between the two calendars. This config is only effective " + "if the writer info (like Spark, Hive) of the Parquet files is unknown.") .version("3.1.0") @@ -4638,7 +4638,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " + "Gregorian) calendar to Proleptic Gregorian calendar when reading Parquet files. " + "When CORRECTED, Spark will not do rebase and read the dates/timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the reading if it sees " + + "When EXCEPTION, Spark will fail the reading if it sees " + "ancient dates/timestamps that are ambiguous between the two calendars. This config is " + "only effective if the writer info (like Spark, Hive) of the Parquet files is unknown. " + "This config influences on reads of the following parquet logical types: DATE, " + @@ -4657,7 +4657,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase dates/timestamps from Proleptic Gregorian calendar " + "to the legacy hybrid (Julian + Gregorian) calendar when writing Avro files. " + "When CORRECTED, Spark will not do rebase and write the dates/timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the writing if it sees " + + "When EXCEPTION, Spark will fail the writing if it sees " + "ancient dates/timestamps that are ambiguous between the two calendars.") .version("3.0.0") .stringConf @@ -4671,7 +4671,7 @@ object SQLConf { .doc("When LEGACY, Spark will rebase dates/timestamps from the legacy hybrid (Julian + " + "Gregorian) calendar to Proleptic Gregorian calendar when reading Avro files. " + "When CORRECTED, Spark will not do rebase and read the dates/timestamps as it is. " + - "When EXCEPTION, which is the default, Spark will fail the reading if it sees " + + "When EXCEPTION, Spark will fail the reading if it sees " + "ancient dates/timestamps that are ambiguous between the two calendars. This config is " + "only effective if the writer info (like Spark, Hive) of the Avro files is unknown.") .version("3.0.0") From 229b1b8654bbe2c7a869566b3c495ffcc432c9ae Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 21 Nov 2024 11:48:41 +0100 Subject: [PATCH 40/58] [SPARK-50375][BUILD] Upgrade `commons-io` to 2.18.0 ### What changes were proposed in this pull request? This PR aims to upgrade `commons-io` from `2.17.0` to `2.18.0`. ### Why are the changes needed? The full release notes: https://commons.apache.org/proper/commons-io/changes-report.html#a2.18.0 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48910 from panbingkun/SPARK-50375. Authored-by: panbingkun Signed-off-by: Max Gekk --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index a866cf7b725c2..4603ae2fc5548 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -44,7 +44,7 @@ commons-compiler/3.1.9//commons-compiler-3.1.9.jar commons-compress/1.27.1//commons-compress-1.27.1.jar commons-crypto/1.1.0//commons-crypto-1.1.0.jar commons-dbcp/1.4//commons-dbcp-1.4.jar -commons-io/2.17.0//commons-io-2.17.0.jar +commons-io/2.18.0//commons-io-2.18.0.jar commons-lang/2.6//commons-lang-2.6.jar commons-lang3/3.17.0//commons-lang3-3.17.0.jar commons-math3/3.6.1//commons-math3-3.6.1.jar diff --git a/pom.xml b/pom.xml index 05c2566ac5f30..9fa0b3cc8a4b7 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,7 @@ 3.0.3 1.17.1 1.27.1 - 2.17.0 + 2.18.0 2.6 From 136c7221fa6a3cb542a7b432dc38032e32859679 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 21 Nov 2024 12:07:31 +0100 Subject: [PATCH 41/58] [SPARK-50334][SQL] Extract common logic for reading the descriptor of PB file ### What changes were proposed in this pull request? The pr aims to - extract `common` logic for `reading the descriptor of PB file` to one place. - at the same time, when using the `from_protobuf` or `to_protobuf` function in `connect-client` and `spark-sql` (or `spark-shell`), the spark error-condition thrown when `the PB file is not found` or `read fails` will be aligned. ### Why are the changes needed? I found that the logic for `reading the descriptor of PB file` is scattered in various places in the `spark code repository`, eg: https://github.com/apache/spark/blob/a01856de20013e5551d385ee000772049a0e1bc0/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala#L37-L48 https://github.com/apache/spark/blob/a01856de20013e5551d385ee000772049a0e1bc0/sql/api/src/main/scala/org/apache/spark/sql/protobuf/functions.scala#L304-L315 https://github.com/apache/spark/blob/a01856de20013e5551d385ee000772049a0e1bc0/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufUtils.scala#L231-L241 - I think we should gather it together to reduce the cost of maintenance. - Align `spark error-condition` to improve consistency in end-user experience. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48874 from panbingkun/SPARK-50334. Authored-by: panbingkun Signed-off-by: Max Gekk --- .../sql/protobuf/utils/ProtobufUtils.scala | 17 -------- .../ProtobufCatalystDataConversionSuite.scala | 7 ++-- .../sql/protobuf/ProtobufFunctionsSuite.scala | 9 ++-- .../sql/protobuf/ProtobufSerdeSuite.scala | 9 ++-- .../apache/spark/sql/protobuf/functions.scala | 26 ++---------- .../apache/spark/sql/util/ProtobufUtils.scala | 41 +++++++++++++++++++ .../toFromProtobufSqlFunctions.scala | 28 ++----------- 7 files changed, 62 insertions(+), 75 deletions(-) create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala diff --git a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufUtils.scala b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufUtils.scala index fee1bcdc9670f..3d7bba7a82e88 100644 --- a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufUtils.scala +++ b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/utils/ProtobufUtils.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.protobuf.utils -import java.io.File -import java.io.FileNotFoundException -import java.nio.file.NoSuchFileException import java.util.Locale import scala.jdk.CollectionConverters._ -import scala.util.control.NonFatal import com.google.protobuf.{DescriptorProtos, Descriptors, InvalidProtocolBufferException, Message} import com.google.protobuf.DescriptorProtos.{FileDescriptorProto, FileDescriptorSet} import com.google.protobuf.Descriptors.{Descriptor, FieldDescriptor} import com.google.protobuf.TypeRegistry -import org.apache.commons.io.FileUtils import org.apache.spark.internal.Logging import org.apache.spark.sql.errors.QueryCompilationErrors @@ -228,18 +223,6 @@ private[sql] object ProtobufUtils extends Logging { } } - def readDescriptorFileContent(filePath: String): Array[Byte] = { - try { - FileUtils.readFileToByteArray(new File(filePath)) - } catch { - case ex: FileNotFoundException => - throw QueryCompilationErrors.cannotFindDescriptorFileError(filePath, ex) - case ex: NoSuchFileException => - throw QueryCompilationErrors.cannotFindDescriptorFileError(filePath, ex) - case NonFatal(ex) => throw QueryCompilationErrors.descriptorParseError(ex) - } - } - private def parseFileDescriptorSet(bytes: Array[Byte]): List[Descriptors.FileDescriptor] = { var fileDescriptorSet: DescriptorProtos.FileDescriptorSet = null try { diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala index ad6a88640140a..abae1d622d3cf 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufCatalystDataConversionSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.protobuf.utils.{ProtobufUtils, SchemaConverters} import org.apache.spark.sql.sources.{EqualTo, Not} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.{ProtobufUtils => CommonProtobufUtils} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ArrayImplicits._ @@ -39,7 +40,7 @@ class ProtobufCatalystDataConversionSuite with ProtobufTestBase { private val testFileDescFile = protobufDescriptorFile("catalyst_types.desc") - private val testFileDesc = ProtobufUtils.readDescriptorFileContent(testFileDescFile) + private val testFileDesc = CommonProtobufUtils.readDescriptorFileContent(testFileDescFile) private val javaClassNamePrefix = "org.apache.spark.sql.protobuf.protos.CatalystTypes$" private def checkResultWithEval( @@ -47,7 +48,7 @@ class ProtobufCatalystDataConversionSuite descFilePath: String, messageName: String, expected: Any): Unit = { - val descBytes = ProtobufUtils.readDescriptorFileContent(descFilePath) + val descBytes = CommonProtobufUtils.readDescriptorFileContent(descFilePath) withClue("(Eval check with Java class name)") { val className = s"$javaClassNamePrefix$messageName" checkEvaluation( @@ -72,7 +73,7 @@ class ProtobufCatalystDataConversionSuite actualSchema: String, badSchema: String): Unit = { - val descBytes = ProtobufUtils.readDescriptorFileContent(descFilePath) + val descBytes = CommonProtobufUtils.readDescriptorFileContent(descFilePath) val binary = CatalystDataToProtobuf(data, actualSchema, Some(descBytes)) intercept[Exception] { diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala index 3eaa91e472c43..44a8339ac1f02 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.protobuf.utils.ProtobufOptions import org.apache.spark.sql.protobuf.utils.ProtobufUtils import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.{ProtobufUtils => CommonProtobufUtils} class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with ProtobufTestBase with Serializable { @@ -40,11 +41,11 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot import testImplicits._ val testFileDescFile = protobufDescriptorFile("functions_suite.desc") - private val testFileDesc = ProtobufUtils.readDescriptorFileContent(testFileDescFile) + private val testFileDesc = CommonProtobufUtils.readDescriptorFileContent(testFileDescFile) private val javaClassNamePrefix = "org.apache.spark.sql.protobuf.protos.SimpleMessageProtos$" val proto2FileDescFile = protobufDescriptorFile("proto2_messages.desc") - val proto2FileDesc = ProtobufUtils.readDescriptorFileContent(proto2FileDescFile) + val proto2FileDesc = CommonProtobufUtils.readDescriptorFileContent(proto2FileDescFile) private val proto2JavaClassNamePrefix = "org.apache.spark.sql.protobuf.protos.Proto2Messages$" private def emptyBinaryDF = Seq(Array[Byte]()).toDF("binary") @@ -467,7 +468,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot test("Handle extra fields : oldProducer -> newConsumer") { val catalystTypesFile = protobufDescriptorFile("catalyst_types.desc") - val descBytes = ProtobufUtils.readDescriptorFileContent(catalystTypesFile) + val descBytes = CommonProtobufUtils.readDescriptorFileContent(catalystTypesFile) val oldProducer = ProtobufUtils.buildDescriptor(descBytes, "oldProducer") val newConsumer = ProtobufUtils.buildDescriptor(descBytes, "newConsumer") @@ -509,7 +510,7 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot test("Handle extra fields : newProducer -> oldConsumer") { val catalystTypesFile = protobufDescriptorFile("catalyst_types.desc") - val descBytes = ProtobufUtils.readDescriptorFileContent(catalystTypesFile) + val descBytes = CommonProtobufUtils.readDescriptorFileContent(catalystTypesFile) val newProducer = ProtobufUtils.buildDescriptor(descBytes, "newProducer") val oldConsumer = ProtobufUtils.buildDescriptor(descBytes, "oldConsumer") diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala index 2737bb9feb3ad..f3bd49e1b24af 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufSerdeSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.protobuf.utils.ProtobufUtils import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.{ProtobufUtils => CommonProtobufUtils} /** * Tests for [[ProtobufSerializer]] and [[ProtobufDeserializer]] with a more specific focus on @@ -37,12 +38,12 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { import ProtoSerdeSuite.MatchType._ private val testFileDescFile = protobufDescriptorFile("serde_suite.desc") - private val testFileDesc = ProtobufUtils.readDescriptorFileContent(testFileDescFile) + private val testFileDesc = CommonProtobufUtils.readDescriptorFileContent(testFileDescFile) private val javaClassNamePrefix = "org.apache.spark.sql.protobuf.protos.SerdeSuiteProtos$" private val proto2DescFile = protobufDescriptorFile("proto2_messages.desc") - private val proto2Desc = ProtobufUtils.readDescriptorFileContent(proto2DescFile) + private val proto2Desc = CommonProtobufUtils.readDescriptorFileContent(proto2DescFile) test("Test basic conversion") { withFieldMatchType { fieldMatch => @@ -215,7 +216,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { val e1 = intercept[AnalysisException] { ProtobufUtils.buildDescriptor( - ProtobufUtils.readDescriptorFileContent(fileDescFile), + CommonProtobufUtils.readDescriptorFileContent(fileDescFile), "SerdeBasicMessage" ) } @@ -225,7 +226,7 @@ class ProtobufSerdeSuite extends SharedSparkSession with ProtobufTestBase { condition = "CANNOT_PARSE_PROTOBUF_DESCRIPTOR") val basicMessageDescWithoutImports = descriptorSetWithoutImports( - ProtobufUtils.readDescriptorFileContent( + CommonProtobufUtils.readDescriptorFileContent( protobufDescriptorFile("basicmessage.desc") ), "BasicMessage" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/protobuf/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/protobuf/functions.scala index ea9e3c429d65a..fab5cdc8de1b7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/protobuf/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/protobuf/functions.scala @@ -16,16 +16,12 @@ */ package org.apache.spark.sql.protobuf -import java.io.FileNotFoundException -import java.nio.file.{Files, NoSuchFileException, Paths} - import scala.jdk.CollectionConverters._ -import scala.util.control.NonFatal import org.apache.spark.annotation.Experimental import org.apache.spark.sql.Column -import org.apache.spark.sql.errors.CompilationErrors import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.util.ProtobufUtils // scalastyle:off: object.name object functions { @@ -51,7 +47,7 @@ object functions { messageName: String, descFilePath: String, options: java.util.Map[String, String]): Column = { - val descriptorFileContent = readDescriptorFileContent(descFilePath) + val descriptorFileContent = ProtobufUtils.readDescriptorFileContent(descFilePath) from_protobuf(data, messageName, descriptorFileContent, options) } @@ -98,7 +94,7 @@ object functions { */ @Experimental def from_protobuf(data: Column, messageName: String, descFilePath: String): Column = { - val fileContent = readDescriptorFileContent(descFilePath) + val fileContent = ProtobufUtils.readDescriptorFileContent(descFilePath) from_protobuf(data, messageName, fileContent) } @@ -226,7 +222,7 @@ object functions { messageName: String, descFilePath: String, options: java.util.Map[String, String]): Column = { - val fileContent = readDescriptorFileContent(descFilePath) + val fileContent = ProtobufUtils.readDescriptorFileContent(descFilePath) to_protobuf(data, messageName, fileContent, options) } @@ -299,18 +295,4 @@ object functions { options: java.util.Map[String, String]): Column = { Column.fnWithOptions("to_protobuf", options.asScala.iterator, data, lit(messageClassName)) } - - // This method is copied from org.apache.spark.sql.protobuf.util.ProtobufUtils - private def readDescriptorFileContent(filePath: String): Array[Byte] = { - try { - Files.readAllBytes(Paths.get(filePath)) - } catch { - case ex: FileNotFoundException => - throw CompilationErrors.cannotFindDescriptorFileError(filePath, ex) - case ex: NoSuchFileException => - throw CompilationErrors.cannotFindDescriptorFileError(filePath, ex) - case NonFatal(ex) => - throw CompilationErrors.descriptorParseError(ex) - } - } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala new file mode 100644 index 0000000000000..11f35ceb060c0 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ProtobufUtils.scala @@ -0,0 +1,41 @@ +/* + * 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.util + +import java.io.{File, FileNotFoundException} +import java.nio.file.NoSuchFileException + +import scala.util.control.NonFatal + +import org.apache.commons.io.FileUtils + +import org.apache.spark.sql.errors.CompilationErrors + +object ProtobufUtils { + def readDescriptorFileContent(filePath: String): Array[Byte] = { + try { + FileUtils.readFileToByteArray(new File(filePath)) + } catch { + case ex: FileNotFoundException => + throw CompilationErrors.cannotFindDescriptorFileError(filePath, ex) + case ex: NoSuchFileException => + throw CompilationErrors.cannotFindDescriptorFileError(filePath, ex) + case NonFatal(ex) => throw CompilationErrors.descriptorParseError(ex) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala index ad9610ea0c78a..96bcf49dbd097 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala @@ -17,37 +17,15 @@ package org.apache.spark.sql.catalyst.expressions -import java.io.File -import java.io.FileNotFoundException -import java.nio.file.NoSuchFileException - -import scala.util.control.NonFatal - -import org.apache.commons.io.FileUtils - import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType} +import org.apache.spark.sql.util.ProtobufUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -object ProtobufHelper { - def readDescriptorFileContent(filePath: String): Array[Byte] = { - try { - FileUtils.readFileToByteArray(new File(filePath)) - } catch { - case ex: FileNotFoundException => - throw new RuntimeException(s"Cannot find descriptor file at path: $filePath", ex) - case ex: NoSuchFileException => - throw new RuntimeException(s"Cannot find descriptor file at path: $filePath", ex) - case NonFatal(ex) => - throw new RuntimeException(s"Failed to read the descriptor file: $filePath", ex) - } - } -} - /** * Converts a binary column of Protobuf format into its corresponding catalyst value. * The Protobuf definition is provided through Protobuf descriptor file. @@ -163,7 +141,7 @@ case class FromProtobuf( } val descFilePathValue: Option[Array[Byte]] = descFilePath.eval() match { case s: UTF8String if s.toString.isEmpty => None - case s: UTF8String => Some(ProtobufHelper.readDescriptorFileContent(s.toString)) + case s: UTF8String => Some(ProtobufUtils.readDescriptorFileContent(s.toString)) case bytes: Array[Byte] if bytes.isEmpty => None case bytes: Array[Byte] => Some(bytes) case null => None @@ -300,7 +278,7 @@ case class ToProtobuf( s.toString } val descFilePathValue: Option[Array[Byte]] = descFilePath.eval() match { - case s: UTF8String => Some(ProtobufHelper.readDescriptorFileContent(s.toString)) + case s: UTF8String => Some(ProtobufUtils.readDescriptorFileContent(s.toString)) case bytes: Array[Byte] => Some(bytes) case null => None } From 2e1c3dc8004b4f003cde8dfae6857f5bef4bb170 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 21 Nov 2024 20:56:59 +0800 Subject: [PATCH 42/58] [SPARK-50087] Robust handling of boolean expressions in CASE WHEN for MsSqlServer and future connectors MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? This PR proposes to propagate the `isPredicate` info in `V2ExpressionBuilder` and wrap the children of CASE WHEN expression (only `Predicate`s) with `IIF(<>, 1, 0)` for MsSqlServer. This is done to force returning an int instead of a boolean, as SqlServer cannot handle boolean expressions as a return type in CASE WHEN. E.g. ```CASE WHEN ... ELSE a = b END``` Old behavior: ```CASE WHEN ... ELSE a = b END = 1``` New behavior: Since in SqlServer a `= 1` is appended to the CASE WHEN, THEN and ELSE blocks must return an int. Therefore the final expression becomes: ```CASE WHEN ... ELSE IIF(a = b, 1, 0) END = 1``` ### Why are the changes needed? A user cannot work with an MsSqlServer data with CASE WHEN clauses or IF clauses if they wish to return a boolean value. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added tests to MsSqlServerIntegrationSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #48621 from andrej-db/SPARK-50087-CaseWhen. Lead-authored-by: Wenchen Fan Co-authored-by: andrej-db Co-authored-by: Andrej Gobeljić Co-authored-by: andrej-gobeljic_data Signed-off-by: Wenchen Fan --- .../jdbc/v2/MsSqlServerIntegrationSuite.scala | 79 +++++++++++++++++++ .../catalyst/util/V2ExpressionBuilder.scala | 6 +- .../apache/spark/sql/jdbc/JdbcDialects.scala | 13 +++ .../spark/sql/jdbc/MsSqlServerDialect.scala | 24 ++++-- 4 files changed, 114 insertions(+), 8 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala index d884ad4c62466..fd7efb1efb764 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala @@ -20,7 +20,11 @@ package org.apache.spark.sql.jdbc.v2 import java.sql.Connection import org.apache.spark.{SparkConf, SparkSQLFeatureNotSupportedException} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog import org.apache.spark.sql.jdbc.MsSQLServerDatabaseOnDocker import org.apache.spark.sql.types._ @@ -37,6 +41,17 @@ import org.apache.spark.tags.DockerTest @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest { + def getExternalEngineQuery(executedPlan: SparkPlan): String = { + getExternalEngineRdd(executedPlan).asInstanceOf[JDBCRDD].getExternalEngineQuery + } + + def getExternalEngineRdd(executedPlan: SparkPlan): RDD[InternalRow] = { + val queryNode = executedPlan.collect { case r: RowDataSourceScanExec => + r + }.head + queryNode.rdd + } + override def excluded: Seq[String] = Seq( "simple scan with OFFSET", "simple scan with LIMIT and OFFSET", @@ -146,4 +161,68 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD |""".stripMargin) assert(df.collect().length == 2) } + + test("SPARK-50087: SqlServer handle booleans in CASE WHEN test") { + val df = sql( + s"""|SELECT * FROM $catalogName.employee + |WHERE CASE WHEN name = 'Legolas' THEN name = 'Elf' ELSE NOT (name = 'Wizard') END + |""".stripMargin + ) + + // scalastyle:off + assert(getExternalEngineQuery(df.queryExecution.executedPlan) == + """SELECT "dept","name","salary","bonus" FROM "employee" WHERE (CASE WHEN ("name" = 'Legolas') THEN IIF(("name" = 'Elf'), 1, 0) ELSE IIF(("name" <> 'Wizard'), 1, 0) END = 1) """ + ) + // scalastyle:on + df.collect() + } + + test("SPARK-50087: SqlServer handle booleans in CASE WHEN with always true test") { + val df = sql( + s"""|SELECT * FROM $catalogName.employee + |WHERE CASE WHEN (name = 'Legolas') THEN (name = 'Elf') ELSE (1=1) END + |""".stripMargin + ) + + // scalastyle:off + assert(getExternalEngineQuery(df.queryExecution.executedPlan) == + """SELECT "dept","name","salary","bonus" FROM "employee" WHERE (CASE WHEN ("name" = 'Legolas') THEN IIF(("name" = 'Elf'), 1, 0) ELSE 1 END = 1) """ + ) + // scalastyle:on + df.collect() + } + + test("SPARK-50087: SqlServer handle booleans in nested CASE WHEN test") { + val df = sql( + s"""|SELECT * FROM $catalogName.employee + |WHERE CASE WHEN (name = 'Legolas') THEN + | CASE WHEN (name = 'Elf') THEN (name = 'Elrond') ELSE (name = 'Gandalf') END + | ELSE (name = 'Sauron') END + |""".stripMargin + ) + + // scalastyle:off + assert(getExternalEngineQuery(df.queryExecution.executedPlan) == + """SELECT "dept","name","salary","bonus" FROM "employee" WHERE (CASE WHEN ("name" = 'Legolas') THEN IIF((CASE WHEN ("name" = 'Elf') THEN IIF(("name" = 'Elrond'), 1, 0) ELSE IIF(("name" = 'Gandalf'), 1, 0) END = 1), 1, 0) ELSE IIF(("name" = 'Sauron'), 1, 0) END = 1) """ + ) + // scalastyle:on + df.collect() + } + + test("SPARK-50087: SqlServer handle non-booleans in nested CASE WHEN test") { + val df = sql( + s"""|SELECT * FROM $catalogName.employee + |WHERE CASE WHEN (name = 'Legolas') THEN + | CASE WHEN (name = 'Elf') THEN 'Elf' ELSE 'Wizard' END + | ELSE 'Sauron' END = name + |""".stripMargin + ) + + // scalastyle:off + assert(getExternalEngineQuery(df.queryExecution.executedPlan) == + """SELECT "dept","name","salary","bonus" FROM "employee" WHERE ("name" IS NOT NULL) AND ((CASE WHEN "name" = 'Legolas' THEN CASE WHEN "name" = 'Elf' THEN 'Elf' ELSE 'Wizard' END ELSE 'Sauron' END) = "name") """ + ) + // scalastyle:on + df.collect() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala index 61a26d7a4fbdf..b0ce2bb4293e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala @@ -221,8 +221,8 @@ class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) extends L case _: BitwiseNot => generateExpressionWithName("~", expr, isPredicate) case caseWhen @ CaseWhen(branches, elseValue) => val conditions = branches.map(_._1).flatMap(generateExpression(_, true)) - val values = branches.map(_._2).flatMap(generateExpression(_)) - val elseExprOpt = elseValue.flatMap(generateExpression(_)) + val values = branches.map(_._2).flatMap(generateExpression(_, isPredicate)) + val elseExprOpt = elseValue.flatMap(generateExpression(_, isPredicate)) if (conditions.length == branches.length && values.length == branches.length && elseExprOpt.size == elseValue.size) { val branchExpressions = conditions.zip(values).flatMap { case (c, v) => @@ -421,7 +421,7 @@ class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) extends L children: Seq[Expression], dataType: DataType, isPredicate: Boolean): Option[V2Expression] = { - val childrenExpressions = children.flatMap(generateExpression(_)) + val childrenExpressions = children.flatMap(generateExpression(_, isPredicate)) if (childrenExpressions.length == children.length) { if (isPredicate && dataType.isInstanceOf[BooleanType]) { Some(new V2Predicate(v2ExpressionName, childrenExpressions.toArray[V2Expression])) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 3bf1390cb664d..81ad1a6d38bbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.connector.catalog.functions.UnboundFunction import org.apache.spark.sql.connector.catalog.index.TableIndex import org.apache.spark.sql.connector.expressions.{Expression, Literal, NamedReference} import org.apache.spark.sql.connector.expressions.aggregate.AggregateFunc +import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.util.V2ExpressionSQLBuilder import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions, JdbcOptionsInWrite, JdbcUtils} @@ -377,6 +378,18 @@ abstract class JdbcDialect extends Serializable with Logging { } private[jdbc] class JDBCSQLBuilder extends V2ExpressionSQLBuilder { + // Some dialects do not support boolean type and this convenient util function is + // provided to generate SQL string without boolean values. + protected def inputToSQLNoBool(input: Expression): String = input match { + case p: Predicate if p.name() == "ALWAYS_TRUE" => "1" + case p: Predicate if p.name() == "ALWAYS_FALSE" => "0" + case p: Predicate => predicateToIntSQL(inputToSQL(p)) + case _ => inputToSQL(input) + } + + protected def predicateToIntSQL(input: String): String = + "CASE WHEN " + input + " THEN 1 ELSE 0 END" + override def visitLiteral(literal: Literal[_]): String = { Option(literal.value()).map(v => compileValue(CatalystTypeConverters.convertToScala(v, literal.dataType())).toString) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index 7d476d43e5c7a..7d339a90db8c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -59,6 +59,8 @@ private case class MsSqlServerDialect() extends JdbcDialect with NoLegacyJDBCErr supportedFunctions.contains(funcName) class MsSqlServerSQLBuilder extends JDBCSQLBuilder { + override protected def predicateToIntSQL(input: String): String = + "IIF(" + input + ", 1, 0)" override def visitSortOrder( sortKey: String, sortDirection: SortDirection, nullOrdering: NullOrdering): String = { (sortDirection, nullOrdering) match { @@ -87,12 +89,24 @@ private case class MsSqlServerDialect() extends JdbcDialect with NoLegacyJDBCErr expr match { case e: Predicate => e.name() match { case "=" | "<>" | "<=>" | "<" | "<=" | ">" | ">=" => - val Array(l, r) = e.children().map { - case p: Predicate => s"CASE WHEN ${inputToSQL(p)} THEN 1 ELSE 0 END" - case o => inputToSQL(o) - } + val Array(l, r) = e.children().map(inputToSQLNoBool) visitBinaryComparison(e.name(), l, r) - case "CASE_WHEN" => visitCaseWhen(expressionsToStringArray(e.children())) + " = 1" + case "CASE_WHEN" => + // Since MsSqlServer cannot handle boolean expressions inside + // a CASE WHEN, it is necessary to convert those to another + // CASE WHEN expression that will return 1 or 0 depending on + // the result. + // Example: + // In: ... CASE WHEN a = b THEN c = d ... END + // Out: ... CASE WHEN a = b THEN CASE WHEN c = d THEN 1 ELSE 0 END ... END = 1 + val stringArray = e.children().grouped(2).flatMap { + case Array(whenExpression, thenExpression) => + Array(inputToSQL(whenExpression), inputToSQLNoBool(thenExpression)) + case Array(elseExpression) => + Array(inputToSQLNoBool(elseExpression)) + }.toArray + + visitCaseWhen(stringArray) + " = 1" case _ => super.build(expr) } case _ => super.build(expr) From 2d09ef22f8adfb562d7da4e192fc5d6201edf34f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 21 Nov 2024 13:07:36 -0800 Subject: [PATCH 43/58] [SPARK-50381][CORE] Support `spark.master.rest.maxThreads` ### What changes were proposed in this pull request? This PR aims to support `spark.master.rest.maxThreads`. ### Why are the changes needed? To provide users a way to control the number of maximum threads of REST API. Previously, Apache Spark uses a default constructor whose value is fixed to `200` always. https://github.com/apache/spark/blob/2e1c3dc8004b4f003cde8dfae6857f5bef4bb170/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala#L94 https://github.com/jetty/jetty.project/blob/5dfc59a691b748796f922208956bd1f2794bcd16/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L118-L121 ### Does this PR introduce _any_ user-facing change? No, the default value of new configuration is identical with the previously-used Jetty's default value. ### How was this patch tested? Pass the CIs with a newly added test case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48921 from dongjoon-hyun/SPARK-50381. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/deploy/rest/RestSubmissionServer.scala | 7 ++++--- .../apache/spark/internal/config/package.scala | 6 ++++++ .../deploy/rest/StandaloneRestSubmitSuite.scala | 16 +++++++++++++++- 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 7efab73726ef8..d3381ef6fb7f1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -33,7 +33,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.config.MASTER_REST_SERVER_FILTERS +import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS} import org.apache.spark.util.Utils /** @@ -63,7 +63,8 @@ private[spark] abstract class RestSubmissionServer( protected val clearRequestServlet: ClearRequestServlet protected val readyzRequestServlet: ReadyzRequestServlet - private var _server: Option[Server] = None + // Visible for testing + private[rest] var _server: Option[Server] = None // A mapping from URL prefixes to servlets that serve them. Exposed for testing. protected val baseContext = s"/${RestSubmissionServer.PROTOCOL_VERSION}/submissions" @@ -91,7 +92,7 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { - val threadPool = new QueuedThreadPool + val threadPool = new QueuedThreadPool(masterConf.get(MASTER_REST_SERVER_MAX_THREADS)) threadPool.setDaemon(true) val server = new Server(threadPool) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index c58c371da20cf..c2b49d164ae3e 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1987,6 +1987,12 @@ package object config { .intConf .createWithDefault(6066) + private[spark] val MASTER_REST_SERVER_MAX_THREADS = ConfigBuilder("spark.master.rest.maxThreads") + .doc("Maximum number of threads to use in the Spark Master REST API Server.") + .version("4.0.0") + .intConf + .createWithDefault(200) + private[spark] val MASTER_REST_SERVER_FILTERS = ConfigBuilder("spark.master.rest.filters") .doc("Comma separated list of filter class names to apply to the Spark Master REST API.") .version("4.0.0") diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 4a05aab01cb50..075a15063c981 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -25,6 +25,7 @@ import java.util.Base64 import scala.collection.mutable import jakarta.servlet.http.HttpServletResponse +import org.eclipse.jetty.util.thread.ThreadPool.SizedThreadPool import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ @@ -33,7 +34,7 @@ import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.DriverState._ import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.config.MASTER_REST_SERVER_FILTERS +import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS} import org.apache.spark.rpc._ import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -545,6 +546,19 @@ class StandaloneRestSubmitSuite extends SparkFunSuite { } } + test("SPARK-50381: Support spark.master.rest.maxThreads") { + val conf = new SparkConf() + val localhost = Utils.localHostName() + val securityManager = new SecurityManager(conf) + rpcEnv = Some(RpcEnv.create("rest-with-maxThreads", localhost, 0, conf, securityManager)) + val fakeMasterRef = rpcEnv.get.setupEndpoint("fake-master", new DummyMaster(rpcEnv.get)) + conf.set(MASTER_REST_SERVER_MAX_THREADS, 2000) + server = Some(new StandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077")) + server.get.start() + val pool = server.get._server.get.getThreadPool.asInstanceOf[SizedThreadPool] + assert(pool.getMaxThreads === 2000) + } + /* --------------------- * | Helper methods | * --------------------- */ From d8a6075dd61748c88733c4964ba37ed2430dc671 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 21 Nov 2024 14:30:19 -0800 Subject: [PATCH 44/58] [SPARK-50380][SQL] ReorderAssociativeOperator should respect the contract in ConstantFolding ### What changes were proposed in this pull request? This PR fixes a long-standing issue in `ReorderAssociativeOperator`. In this rule, we flatten the Add/Multiply nodes, and combine the foldable operands into a single Add/Multiply, then evaluate it into a literal. This is fine normally, but we added a new contract in `ConstantFolding` with https://github.com/apache/spark/pull/36468 , due to the introduction of ANSI mode and we don't want to fail eagerly for expressions within conditional branches. `ReorderAssociativeOperator` does not follow this contract. The solution in this PR is to leave the expression evaluation to `ConstantFolding`. `ReorderAssociativeOperator` should only match literals. This makes sure that the early expression evaluation follows all the contracts in `ConstantFolding`. ### Why are the changes needed? Avoid failing the query which should not fail. This also fixes a regression caused by https://github.com/apache/spark/pull/48395 , which does not introduce the bug, but makes the bug more likely to happen. ### Does this PR introduce _any_ user-facing change? Yes, failed queries can run now. ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #48918 from cloud-fan/error. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/expressions.scala | 36 ++++++++++--------- .../ReorderAssociativeOperatorSuite.scala | 20 +++++++++-- 2 files changed, 37 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 3eb7eb6e6b2e8..754fea85ec6d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -249,6 +249,11 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { case _ => ExpressionSet(Seq.empty) } + private def isSameInteger(expr: Expression, value: Int): Boolean = expr match { + case l: Literal => l.value == value + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( _.containsPattern(BINARY_ARITHMETIC), ruleId) { case q: LogicalPlan => @@ -259,32 +264,31 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { val groupingExpressionSet = collectGroupingExpressions(q) q.transformExpressionsDownWithPruning(_.containsPattern(BINARY_ARITHMETIC)) { case a @ Add(_, _, f) if a.deterministic && a.dataType.isInstanceOf[IntegralType] => - val (foldables, others) = flattenAdd(a, groupingExpressionSet).partition(_.foldable) - if (foldables.nonEmpty) { - val foldableExpr = foldables.reduce((x, y) => Add(x, y, f)) - val foldableValue = foldableExpr.eval(EmptyRow) + val (literals, others) = flattenAdd(a, groupingExpressionSet) + .partition(_.isInstanceOf[Literal]) + if (literals.nonEmpty) { + val literalExpr = literals.reduce((x, y) => Add(x, y, f)) if (others.isEmpty) { - Literal.create(foldableValue, a.dataType) - } else if (foldableValue == 0) { + literalExpr + } else if (isSameInteger(literalExpr, 0)) { others.reduce((x, y) => Add(x, y, f)) } else { - Add(others.reduce((x, y) => Add(x, y, f)), Literal.create(foldableValue, a.dataType), f) + Add(others.reduce((x, y) => Add(x, y, f)), literalExpr, f) } } else { a } case m @ Multiply(_, _, f) if m.deterministic && m.dataType.isInstanceOf[IntegralType] => - val (foldables, others) = flattenMultiply(m, groupingExpressionSet).partition(_.foldable) - if (foldables.nonEmpty) { - val foldableExpr = foldables.reduce((x, y) => Multiply(x, y, f)) - val foldableValue = foldableExpr.eval(EmptyRow) - if (others.isEmpty || (foldableValue == 0 && !m.nullable)) { - Literal.create(foldableValue, m.dataType) - } else if (foldableValue == 1) { + val (literals, others) = flattenMultiply(m, groupingExpressionSet) + .partition(_.isInstanceOf[Literal]) + if (literals.nonEmpty) { + val literalExpr = literals.reduce((x, y) => Multiply(x, y, f)) + if (others.isEmpty || (isSameInteger(literalExpr, 0) && !m.nullable)) { + literalExpr + } else if (isSameInteger(literalExpr, 1)) { others.reduce((x, y) => Multiply(x, y, f)) } else { - Multiply(others.reduce((x, y) => Multiply(x, y, f)), - Literal.create(foldableValue, m.dataType), f) + Multiply(others.reduce((x, y) => Multiply(x, y, f)), literalExpr, f) } } else { m diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala index 9090e0c7fc104..7733e58547fe0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala @@ -29,7 +29,8 @@ class ReorderAssociativeOperatorSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("ReorderAssociativeOperator", Once, + Batch("ReorderAssociativeOperator", FixedPoint(10), + ConstantFolding, ReorderAssociativeOperator) :: Nil } @@ -44,7 +45,7 @@ class ReorderAssociativeOperatorSuite extends PlanTest { ($"b" + 1) * 2 * 3 * 4, $"a" + 1 + $"b" + 2 + $"c" + 3, $"a" + 1 + $"b" * 2 + $"c" + 3, - Rand(0) * 1 * 2 * 3 * 4) + Rand(0) * 1.0 * 2.0 * 3.0 * 4.0) val optimized = Optimize.execute(originalQuery.analyze) @@ -56,7 +57,7 @@ class ReorderAssociativeOperatorSuite extends PlanTest { (($"b" + 1) * 24).as("((((b + 1) * 2) * 3) * 4)"), ($"a" + $"b" + $"c" + 6).as("(((((a + 1) + b) + 2) + c) + 3)"), ($"a" + $"b" * 2 + $"c" + 4).as("((((a + 1) + (b * 2)) + c) + 3)"), - Rand(0) * 1 * 2 * 3 * 4) + Rand(0) * 1.0 * 2.0 * 3.0 * 4.0) .analyze comparePlans(optimized, correctAnswer) @@ -106,4 +107,17 @@ class ReorderAssociativeOperatorSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("SPARK-50380: conditional branches with error expression") { + val originalQuery1 = testRelation.select(If($"a" === 1, 1L, Literal(1).div(0) + $"b")).analyze + val optimized1 = Optimize.execute(originalQuery1) + comparePlans(optimized1, originalQuery1) + + val originalQuery2 = testRelation.select( + If($"a" === 1, 1, ($"b" + Literal(Int.MaxValue)) + 1).as("col")).analyze + val optimized2 = Optimize.execute(originalQuery2) + val correctAnswer2 = testRelation.select( + If($"a" === 1, 1, $"b" + (Literal(Int.MaxValue) + 1)).as("col")).analyze + comparePlans(optimized2, correctAnswer2) + } } From 6881ec0ebfab36a59296157458a928331c1d5103 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Fri, 22 Nov 2024 09:23:34 +0900 Subject: [PATCH 45/58] [SPARK-50366][SQL] Isolate user-defined tags on thread level for SparkSession in Classic ### What changes were proposed in this pull request? This PR changes the implementation of user-provided tags to be thread-local, so that tags added by two threads to the same SparkSession do not interfere with each other. Overlaps (from the `SparkContext` perspective) are avoided by introducing a thread-local random UUID which is attached to all tags in the same thread. ### Why are the changes needed? To make tags isolated per thread. ### Does this PR introduce _any_ user-facing change? Yes, user-provided tags are now isolated on the session level. ### How was this patch tested? Local test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48906 from xupefei/thread-isolated-tags. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/SparkSession.scala | 47 +++++--- .../spark/sql/execution/SQLExecution.scala | 2 +- ...essionJobTaggingAndCancellationSuite.scala | 102 +++++++++++++++--- 3 files changed, 120 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index afc0a2d7df604..a7f85db12b214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql import java.net.URI import java.nio.file.Paths import java.util.{ServiceLoader, UUID} -import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.mutable import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag @@ -133,14 +133,34 @@ class SparkSession private( /** Tag to mark all jobs owned by this session. */ private[sql] lazy val sessionJobTag = s"spark-session-$sessionUUID" + /** + * A UUID that is unique on the thread level. Used by managedJobTags to make sure that a same + * tag from two threads does not overlap in the underlying SparkContext/SQLExecution. + */ + private[sql] lazy val threadUuid = new InheritableThreadLocal[String] { + override def childValue(parent: String): String = parent + + override def initialValue(): String = UUID.randomUUID().toString + } + /** * A map to hold the mapping from user-defined tags to the real tags attached to Jobs. - * Real tag have the current session ID attached: `"tag1" -> s"spark-session-$sessionUUID-tag1"`. + * Real tag have the current session ID attached: + * tag1 -> spark-session-$sessionUUID-thread-$threadUuid-tag1 + * */ @transient - private[sql] lazy val managedJobTags: ConcurrentHashMap[String, String] = { - new ConcurrentHashMap(parentManagedJobTags.asJava) - } + private[sql] lazy val managedJobTags = new InheritableThreadLocal[mutable.Map[String, String]] { + override def childValue(parent: mutable.Map[String, String]): mutable.Map[String, String] = { + // Note: make a clone such that changes in the parent tags aren't reflected in + // those of the children threads. + parent.clone() + } + + override def initialValue(): mutable.Map[String, String] = { + mutable.Map(parentManagedJobTags.toSeq: _*) + } + } /** @inheritdoc */ def version: String = SPARK_VERSION @@ -243,10 +263,10 @@ class SparkSession private( Some(sessionState), extensions, Map.empty, - managedJobTags.asScala.toMap) + managedJobTags.get().toMap) result.sessionState // force copy of SessionState result.sessionState.artifactManager // force copy of ArtifactManager and its resources - result.managedJobTags // force copy of userDefinedToRealTagsMap + result.managedJobTags // force copy of managedJobTags result } @@ -550,17 +570,17 @@ class SparkSession private( /** @inheritdoc */ override def addTag(tag: String): Unit = { SparkContext.throwIfInvalidTag(tag) - managedJobTags.put(tag, s"spark-session-$sessionUUID-$tag") + managedJobTags.get().put(tag, s"spark-session-$sessionUUID-thread-${threadUuid.get()}-$tag") } /** @inheritdoc */ - override def removeTag(tag: String): Unit = managedJobTags.remove(tag) + override def removeTag(tag: String): Unit = managedJobTags.get().remove(tag) /** @inheritdoc */ - override def getTags(): Set[String] = managedJobTags.keys().asScala.toSet + override def getTags(): Set[String] = managedJobTags.get().keySet.toSet /** @inheritdoc */ - override def clearTags(): Unit = managedJobTags.clear() + override def clearTags(): Unit = managedJobTags.get().clear() /** * Request to interrupt all currently running SQL operations of this session. @@ -589,9 +609,8 @@ class SparkSession private( * @since 4.0.0 */ override def interruptTag(tag: String): Seq[String] = { - val realTag = managedJobTags.get(tag) - if (realTag == null) return Seq.empty - doInterruptTag(realTag, s"part of cancelled job tags $tag") + val realTag = managedJobTags.get().get(tag) + realTag.map(doInterruptTag(_, s"part of cancelled job tags $tag")).getOrElse(Seq.empty) } private def doInterruptTag(tag: String, reason: String): Seq[String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index e805aabe013cf..242149010ceef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -261,7 +261,7 @@ object SQLExecution extends Logging { } private[sql] def withSessionTagsApplied[T](sparkSession: SparkSession)(block: => T): T = { - val allTags = sparkSession.managedJobTags.values().asScala.toSet + sparkSession.sessionJobTag + val allTags = sparkSession.managedJobTags.get().values.toSet + sparkSession.sessionJobTag sparkSession.sparkContext.addJobTags(allTags) try { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionJobTaggingAndCancellationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionJobTaggingAndCancellationSuite.scala index 1ac51b408301a..89500fe51f3ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionJobTaggingAndCancellationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionJobTaggingAndCancellationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.util.concurrent.{ConcurrentHashMap, Semaphore, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, Executors, Semaphore, TimeUnit} import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.{ExecutionContext, Future} @@ -100,13 +100,14 @@ class SparkSessionJobTaggingAndCancellationSuite assert(sem.tryAcquire(1, 1, TimeUnit.MINUTES)) val activeJobsFuture = - session.sparkContext.cancelJobsWithTagWithFuture(session.managedJobTags.get("one"), "reason") + session.sparkContext.cancelJobsWithTagWithFuture( + session.managedJobTags.get()("one"), "reason") val activeJob = ThreadUtils.awaitResult(activeJobsFuture, 60.seconds).head val actualTags = activeJob.properties.getProperty(SparkContext.SPARK_JOB_TAGS) .split(SparkContext.SPARK_JOB_TAGS_SEP) assert(actualTags.toSet == Set( session.sessionJobTag, - s"${session.sessionJobTag}-one", + s"${session.sessionJobTag}-thread-${session.threadUuid.get()}-one", SQLExecution.executionIdJobTag( session, activeJob.properties.get(SQLExecution.EXECUTION_ROOT_ID_KEY).asInstanceOf[String].toLong))) @@ -118,12 +119,12 @@ class SparkSessionJobTaggingAndCancellationSuite val globalSession = SparkSession.builder().sparkContext(sc).getOrCreate() var (sessionA, sessionB, sessionC): (SparkSession, SparkSession, SparkSession) = (null, null, null) + var (threadUuidA, threadUuidB, threadUuidC): (String, String, String) = (null, null, null) // global ExecutionContext has only 2 threads in Apache Spark CI // create own thread pool for four Futures used in this test - val numThreads = 3 - val fpool = ThreadUtils.newForkJoinPool("job-tags-test-thread-pool", numThreads) - val executionContext = ExecutionContext.fromExecutorService(fpool) + val threadPool = Executors.newFixedThreadPool(3) + implicit val ec: ExecutionContext = ExecutionContext.fromExecutor(threadPool) try { // Add a listener to release the semaphore once jobs are launched. @@ -143,28 +144,50 @@ class SparkSessionJobTaggingAndCancellationSuite } }) + var realTagOneForSessionA: String = null + var childThread: Thread = null + val childThreadLock = new Semaphore(0) + // Note: since tags are added in the Future threads, they don't need to be cleared in between. val jobA = Future { sessionA = globalSession.cloneSession() import globalSession.implicits._ + threadUuidA = sessionA.threadUuid.get() assert(sessionA.getTags() == Set()) sessionA.addTag("two") assert(sessionA.getTags() == Set("two")) sessionA.clearTags() // check that clearing all tags works assert(sessionA.getTags() == Set()) sessionA.addTag("one") + realTagOneForSessionA = sessionA.managedJobTags.get()("one") + assert(realTagOneForSessionA == + s"${sessionA.sessionJobTag}-thread-${sessionA.threadUuid.get()}-one") assert(sessionA.getTags() == Set("one")) + + // Create a child thread which inherits thread-local variables and tries to interrupt + // the job started from the parent thread. The child thread is blocked until the main + // thread releases the lock. + childThread = new Thread { + override def run(): Unit = { + assert(childThreadLock.tryAcquire(1, 20, TimeUnit.SECONDS)) + assert(sessionA.getTags() == Set("one")) + assert(sessionA.interruptTag("one").size == 1) + } + } + childThread.start() try { sessionA.range(1, 10000).map { i => Thread.sleep(100); i }.count() } finally { + childThread.interrupt() sessionA.clearTags() // clear for the case of thread reuse by another Future } - }(executionContext) + } val jobB = Future { sessionB = globalSession.cloneSession() import globalSession.implicits._ + threadUuidB = sessionB.threadUuid.get() assert(sessionB.getTags() == Set()) sessionB.addTag("one") sessionB.addTag("two") @@ -176,11 +199,12 @@ class SparkSessionJobTaggingAndCancellationSuite } finally { sessionB.clearTags() // clear for the case of thread reuse by another Future } - }(executionContext) + } val jobC = Future { sessionC = globalSession.cloneSession() import globalSession.implicits._ + threadUuidC = sessionC.threadUuid.get() sessionC.addTag("foo") sessionC.removeTag("foo") assert(sessionC.getTags() == Set()) // check that remove works removing the last tag @@ -190,12 +214,13 @@ class SparkSessionJobTaggingAndCancellationSuite } finally { sessionC.clearTags() // clear for the case of thread reuse by another Future } - }(executionContext) + } // Block until four jobs have started. assert(sem.tryAcquire(3, 1, TimeUnit.MINUTES)) // Tags are applied + def realUserTag(s: String, t: String, ta: String): String = s"spark-session-$s-thread-$t-$ta" assert(jobProperties.size == 3) for (ss <- Seq(sessionA, sessionB, sessionC)) { val jobProperty = jobProperties.values().asScala.filter(_.get(SparkContext.SPARK_JOB_TAGS) @@ -207,15 +232,17 @@ class SparkSessionJobTaggingAndCancellationSuite val executionRootIdTag = SQLExecution.executionIdJobTag( ss, jobProperty.head.get(SQLExecution.EXECUTION_ROOT_ID_KEY).asInstanceOf[String].toLong) - val userTagsPrefix = s"spark-session-${ss.sessionUUID}-" ss match { case s if s == sessionA => assert(tags.toSet == Set( - s.sessionJobTag, executionRootIdTag, s"${userTagsPrefix}one")) + s.sessionJobTag, executionRootIdTag, realUserTag(s.sessionUUID, threadUuidA, "one"))) case s if s == sessionB => assert(tags.toSet == Set( - s.sessionJobTag, executionRootIdTag, s"${userTagsPrefix}one", s"${userTagsPrefix}two")) + s.sessionJobTag, + executionRootIdTag, + realUserTag(s.sessionUUID, threadUuidB, "one"), + realUserTag(s.sessionUUID, threadUuidB, "two"))) case s if s == sessionC => assert(tags.toSet == Set( - s.sessionJobTag, executionRootIdTag, s"${userTagsPrefix}boo")) + s.sessionJobTag, executionRootIdTag, realUserTag(s.sessionUUID, threadUuidC, "boo"))) } } @@ -239,8 +266,10 @@ class SparkSessionJobTaggingAndCancellationSuite assert(sem.tryAcquire(1, 1, TimeUnit.MINUTES)) assert(jobEnded.intValue == 1) - // Another job cancelled - assert(sessionA.interruptTag("one").size == 1) + // Another job cancelled. The next line cancels nothing because we're now in another thread. + // The real cancel is done through unblocking a child thread, which is waiting for a lock + assert(sessionA.interruptTag("one").isEmpty) + childThreadLock.release() val eA = intercept[SparkException] { ThreadUtils.awaitResult(jobA, 1.minute) }.getCause @@ -257,7 +286,48 @@ class SparkSessionJobTaggingAndCancellationSuite assert(sem.tryAcquire(1, 1, TimeUnit.MINUTES)) assert(jobEnded.intValue == 3) } finally { - fpool.shutdownNow() + threadPool.shutdownNow() + } + } + + test("Tags are isolated in multithreaded environment") { + // Custom thread pool for multi-threaded testing + val threadPool = Executors.newFixedThreadPool(2) + implicit val ec: ExecutionContext = ExecutionContext.fromExecutor(threadPool) + + val session = SparkSession.builder().master("local").getOrCreate() + @volatile var output1: Set[String] = null + @volatile var output2: Set[String] = null + + def tag1(): Unit = { + session.addTag("tag1") + output1 = session.getTags() + } + + def tag2(): Unit = { + session.addTag("tag2") + output2 = session.getTags() + } + + try { + // Run tasks in separate threads + val future1 = Future { + tag1() + } + val future2 = Future { + tag2() + } + + // Wait for threads to complete + ThreadUtils.awaitResult(Future.sequence(Seq(future1, future2)), 1.minute) + + // Assert outputs + assert(output1 != null) + assert(output1 == Set("tag1")) + assert(output2 != null) + assert(output2 == Set("tag2")) + } finally { + threadPool.shutdownNow() } } } From 5d1f585eba694a446cf3f9494b80c485b6a176b4 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 21 Nov 2024 17:31:59 -0800 Subject: [PATCH 46/58] [SPARK-50383][CORE] Support `Virtual Threads` in REST Submission API ### What changes were proposed in this pull request? This PR aims to support [JEP-444: Virtual Threads](https://openjdk.org/jeps/444) in REST Submission API for Apache Spark 4.0.0. - https://openjdk.org/jeps/425: Virtual Threads (Preview) in Java 19 - https://openjdk.org/jeps/436: Virtual Threads (Second Preview) in Java 20 - https://openjdk.org/jeps/444: Virtual Threads in Java 21 ### Why are the changes needed? `Virtual Threads` becomes an official feature in Java 21. - https://docs.oracle.com/en/java/javase/21/core/virtual-threads.html > Virtual threads are lightweight threads that reduce the effort of writing, maintaining, and debugging high-throughput concurrent applications. In addition, `Jetty 11` supports `Virtual Threads`. - https://jetty.org/docs/jetty/11/programming-guide/arch/threads.html#thread-pool-virtual-threads We had better provide a way to use it via a new option `spark.master.rest.virtualThread.enabled`, which is enabled in Java 21 runtime. Please note that this aims for high-throughput concurrency. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48923 from dongjoon-hyun/SPARK-50383. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../deploy/rest/RestSubmissionServer.scala | 9 ++++++++- .../spark/internal/config/package.scala | 7 +++++++ .../rest/StandaloneRestSubmitSuite.scala | 20 ++++++++++++++++++- 3 files changed, 34 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index d3381ef6fb7f1..877349da18dd0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.rest import java.util.EnumSet +import java.util.concurrent.{Executors, ExecutorService} import scala.io.Source @@ -33,7 +34,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS} +import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS, MASTER_REST_SERVER_VIRTUAL_THREADS} import org.apache.spark.util.Utils /** @@ -93,6 +94,12 @@ private[spark] abstract class RestSubmissionServer( */ private def doStart(startPort: Int): (Server, Int) = { val threadPool = new QueuedThreadPool(masterConf.get(MASTER_REST_SERVER_MAX_THREADS)) + if (Utils.isJavaVersionAtLeast21 && masterConf.get(MASTER_REST_SERVER_VIRTUAL_THREADS)) { + val newVirtualThreadPerTaskExecutor = + classOf[Executors].getMethod("newVirtualThreadPerTaskExecutor") + val service = newVirtualThreadPerTaskExecutor.invoke(null).asInstanceOf[ExecutorService] + threadPool.setVirtualThreadsExecutor(service) + } threadPool.setDaemon(true) val server = new Server(threadPool) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index c2b49d164ae3e..324ef701c4266 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -2000,6 +2000,13 @@ package object config { .toSequence .createWithDefault(Nil) + private[spark] val MASTER_REST_SERVER_VIRTUAL_THREADS = + ConfigBuilder("spark.master.rest.virtualThread.enabled") + .doc("If true, Spark master tries to use Java 21 virtual thread for REST API.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") .version("1.1.0") .intConf diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 075a15063c981..a155e4cc3ac90 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -25,6 +25,7 @@ import java.util.Base64 import scala.collection.mutable import jakarta.servlet.http.HttpServletResponse +import org.eclipse.jetty.util.thread.QueuedThreadPool import org.eclipse.jetty.util.thread.ThreadPool.SizedThreadPool import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ @@ -34,7 +35,7 @@ import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.DriverState._ import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS} +import org.apache.spark.internal.config.{MASTER_REST_SERVER_FILTERS, MASTER_REST_SERVER_MAX_THREADS, MASTER_REST_SERVER_VIRTUAL_THREADS} import org.apache.spark.rpc._ import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -559,6 +560,23 @@ class StandaloneRestSubmitSuite extends SparkFunSuite { assert(pool.getMaxThreads === 2000) } + test("SPARK-50383: Support spark.master.rest.virtualThread.enabled") { + val conf = new SparkConf() + val localhost = Utils.localHostName() + val securityManager = new SecurityManager(conf) + rpcEnv = Some(RpcEnv.create("rest-with-virtualThreads", localhost, 0, conf, securityManager)) + val fakeMasterRef = rpcEnv.get.setupEndpoint("fake-master", new DummyMaster(rpcEnv.get)) + conf.set(MASTER_REST_SERVER_VIRTUAL_THREADS, true) + server = Some(new StandaloneRestServer(localhost, 0, conf, fakeMasterRef, "spark://fake:7077")) + server.get.start() + val pool = server.get._server.get.getThreadPool.asInstanceOf[QueuedThreadPool] + if (Utils.isJavaVersionAtLeast21) { + assert(pool.getVirtualThreadsExecutor != null) + } else { + assert(pool.getVirtualThreadsExecutor == null) + } + } + /* --------------------- * | Helper methods | * --------------------- */ From 190c50451333c5f6be349defcac1ad1983632935 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 22 Nov 2024 10:22:30 +0800 Subject: [PATCH 47/58] [SPARK-50388][PYTHON][TESTS] Further centralize import checks ### What changes were proposed in this pull request? Further centralized import checks: 1, move `have_xxx` from `sqlutils.py/pandasutils.py/xxx` to `utils.py`; 2, but still keep `have_pandas` and `have_pyarrow` in `sqlutils.py`, by importing them from `utils.py`, because there are too many usage places ### Why are the changes needed? simplify the import checks, e.g. `have_plotly` has been defined in multiple places ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #48926 from zhengruifeng/py_dep_2. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/pandas/tests/io/test_io.py | 7 +--- .../tests/plot/test_frame_plot_matplotlib.py | 8 +--- .../tests/plot/test_frame_plot_plotly.py | 8 +--- .../pandas/tests/plot/test_series_plot.py | 2 +- .../tests/plot/test_series_plot_matplotlib.py | 8 +--- .../tests/plot/test_series_plot_plotly.py | 8 +--- .../pandas/tests/series/test_conversion.py | 2 +- python/pyspark/sql/metrics.py | 6 +-- .../sql/tests/connect/test_df_debug.py | 7 +--- .../pyspark/sql/tests/plot/test_frame_plot.py | 4 +- .../sql/tests/plot/test_frame_plot_plotly.py | 4 +- python/pyspark/testing/pandasutils.py | 23 ----------- python/pyspark/testing/sqlutils.py | 41 +++++-------------- python/pyspark/testing/utils.py | 33 +++++++++++++++ 14 files changed, 64 insertions(+), 97 deletions(-) diff --git a/python/pyspark/pandas/tests/io/test_io.py b/python/pyspark/pandas/tests/io/test_io.py index d4e61319f229c..6fbdc366dd76a 100644 --- a/python/pyspark/pandas/tests/io/test_io.py +++ b/python/pyspark/pandas/tests/io/test_io.py @@ -22,12 +22,9 @@ import pandas as pd from pyspark import pandas as ps -from pyspark.testing.pandasutils import ( - have_tabulate, - PandasOnSparkTestCase, - tabulate_requirement_message, -) +from pyspark.testing.pandasutils import PandasOnSparkTestCase from pyspark.testing.sqlutils import SQLTestUtils +from pyspark.testing.utils import have_tabulate, tabulate_requirement_message # This file contains test cases for 'Serialization / IO / Conversion' diff --git a/python/pyspark/pandas/tests/plot/test_frame_plot_matplotlib.py b/python/pyspark/pandas/tests/plot/test_frame_plot_matplotlib.py index 365d34b1f550e..1d63cafe19b42 100644 --- a/python/pyspark/pandas/tests/plot/test_frame_plot_matplotlib.py +++ b/python/pyspark/pandas/tests/plot/test_frame_plot_matplotlib.py @@ -24,12 +24,8 @@ from pyspark import pandas as ps from pyspark.pandas.config import set_option, reset_option -from pyspark.testing.pandasutils import ( - have_matplotlib, - matplotlib_requirement_message, - PandasOnSparkTestCase, - TestUtils, -) +from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils +from pyspark.testing.utils import have_matplotlib, matplotlib_requirement_message if have_matplotlib: import matplotlib diff --git a/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py b/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py index 8d197649aaebe..5308932573330 100644 --- a/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py +++ b/python/pyspark/pandas/tests/plot/test_frame_plot_plotly.py @@ -23,12 +23,8 @@ from pyspark import pandas as ps from pyspark.pandas.config import set_option, reset_option -from pyspark.testing.pandasutils import ( - have_plotly, - plotly_requirement_message, - PandasOnSparkTestCase, - TestUtils, -) +from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils +from pyspark.testing.utils import have_plotly, plotly_requirement_message from pyspark.pandas.utils import name_like_string if have_plotly: diff --git a/python/pyspark/pandas/tests/plot/test_series_plot.py b/python/pyspark/pandas/tests/plot/test_series_plot.py index 6e0bdd232fc41..61d114f37b0e8 100644 --- a/python/pyspark/pandas/tests/plot/test_series_plot.py +++ b/python/pyspark/pandas/tests/plot/test_series_plot.py @@ -22,7 +22,7 @@ from pyspark import pandas as ps from pyspark.pandas.plot import PandasOnSparkPlotAccessor, BoxPlotBase -from pyspark.testing.pandasutils import have_plotly, plotly_requirement_message +from pyspark.testing.utils import have_plotly, plotly_requirement_message class SeriesPlotTestsMixin: diff --git a/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py b/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py index c98c1aeea04e7..0fdcbc9d748e0 100644 --- a/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py +++ b/python/pyspark/pandas/tests/plot/test_series_plot_matplotlib.py @@ -24,12 +24,8 @@ from pyspark import pandas as ps from pyspark.pandas.config import set_option, reset_option -from pyspark.testing.pandasutils import ( - have_matplotlib, - matplotlib_requirement_message, - PandasOnSparkTestCase, - TestUtils, -) +from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils +from pyspark.testing.utils import have_matplotlib, matplotlib_requirement_message if have_matplotlib: import matplotlib diff --git a/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py b/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py index 1aa175f9308a1..8123af26dbf4b 100644 --- a/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py +++ b/python/pyspark/pandas/tests/plot/test_series_plot_plotly.py @@ -24,12 +24,8 @@ from pyspark import pandas as ps from pyspark.pandas.config import set_option, reset_option from pyspark.pandas.utils import name_like_string -from pyspark.testing.pandasutils import ( - have_plotly, - plotly_requirement_message, - PandasOnSparkTestCase, - TestUtils, -) +from pyspark.testing.pandasutils import PandasOnSparkTestCase, TestUtils +from pyspark.testing.utils import have_plotly, plotly_requirement_message if have_plotly: from plotly import express diff --git a/python/pyspark/pandas/tests/series/test_conversion.py b/python/pyspark/pandas/tests/series/test_conversion.py index 71ae858631d4d..7711d05abd76d 100644 --- a/python/pyspark/pandas/tests/series/test_conversion.py +++ b/python/pyspark/pandas/tests/series/test_conversion.py @@ -21,7 +21,7 @@ from pyspark import pandas as ps from pyspark.testing.pandasutils import PandasOnSparkTestCase from pyspark.testing.sqlutils import SQLTestUtils -from pyspark.testing.pandasutils import have_tabulate, tabulate_requirement_message +from pyspark.testing.utils import have_tabulate, tabulate_requirement_message class SeriesConversionMixin: diff --git a/python/pyspark/sql/metrics.py b/python/pyspark/sql/metrics.py index 0f4142e91b256..4ab9b041e3135 100644 --- a/python/pyspark/sql/metrics.py +++ b/python/pyspark/sql/metrics.py @@ -21,10 +21,10 @@ from pyspark.errors import PySparkValueError if TYPE_CHECKING: - from pyspark.testing.connectutils import have_graphviz - - if have_graphviz: + try: import graphviz # type: ignore + except ImportError: + pass class ObservedMetrics(abc.ABC): diff --git a/python/pyspark/sql/tests/connect/test_df_debug.py b/python/pyspark/sql/tests/connect/test_df_debug.py index 8a4ec68fda844..40b6a072e9127 100644 --- a/python/pyspark/sql/tests/connect/test_df_debug.py +++ b/python/pyspark/sql/tests/connect/test_df_debug.py @@ -17,12 +17,9 @@ import unittest -from pyspark.testing.connectutils import ( - should_test_connect, - have_graphviz, - graphviz_requirement_message, -) from pyspark.sql.tests.connect.test_connect_basic import SparkConnectSQLTestCase +from pyspark.testing.connectutils import should_test_connect +from pyspark.testing.utils import have_graphviz, graphviz_requirement_message if should_test_connect: from pyspark.sql.connect.dataframe import DataFrame diff --git a/python/pyspark/sql/tests/plot/test_frame_plot.py b/python/pyspark/sql/tests/plot/test_frame_plot.py index 3221a408d153d..c37aef5f7c94f 100644 --- a/python/pyspark/sql/tests/plot/test_frame_plot.py +++ b/python/pyspark/sql/tests/plot/test_frame_plot.py @@ -18,8 +18,8 @@ import unittest from pyspark.errors import PySparkValueError from pyspark.sql import Row -from pyspark.testing.sqlutils import ( - ReusedSQLTestCase, +from pyspark.testing.sqlutils import ReusedSQLTestCase +from pyspark.testing.utils import ( have_plotly, plotly_requirement_message, have_pandas, diff --git a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py index 84a9c2aa01706..fd264c3488823 100644 --- a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py +++ b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py @@ -19,8 +19,8 @@ from datetime import datetime from pyspark.errors import PySparkTypeError, PySparkValueError -from pyspark.testing.sqlutils import ( - ReusedSQLTestCase, +from pyspark.testing.sqlutils import ReusedSQLTestCase +from pyspark.testing.utils import ( have_plotly, plotly_requirement_message, have_pandas, diff --git a/python/pyspark/testing/pandasutils.py b/python/pyspark/testing/pandasutils.py index 10e8ce6f69af3..09d3ffb09708f 100644 --- a/python/pyspark/testing/pandasutils.py +++ b/python/pyspark/testing/pandasutils.py @@ -23,29 +23,6 @@ import decimal from typing import Any, Union -tabulate_requirement_message = None -try: - from tabulate import tabulate -except ImportError as e: - # If tabulate requirement is not satisfied, skip related tests. - tabulate_requirement_message = str(e) -have_tabulate = tabulate_requirement_message is None - -matplotlib_requirement_message = None -try: - import matplotlib -except ImportError as e: - # If matplotlib requirement is not satisfied, skip related tests. - matplotlib_requirement_message = str(e) -have_matplotlib = matplotlib_requirement_message is None - -plotly_requirement_message = None -try: - import plotly -except ImportError as e: - # If plotly requirement is not satisfied, skip related tests. - plotly_requirement_message = str(e) -have_plotly = plotly_requirement_message is None try: from pyspark.sql.pandas.utils import require_minimum_pandas_version diff --git a/python/pyspark/testing/sqlutils.py b/python/pyspark/testing/sqlutils.py index c833abfb805dc..e5464257422ae 100644 --- a/python/pyspark/testing/sqlutils.py +++ b/python/pyspark/testing/sqlutils.py @@ -22,23 +22,17 @@ import tempfile from contextlib import contextmanager -pandas_requirement_message = None -try: - from pyspark.sql.pandas.utils import require_minimum_pandas_version - - require_minimum_pandas_version() -except ImportError as e: - # If Pandas version requirement is not satisfied, skip related tests. - pandas_requirement_message = str(e) - -pyarrow_requirement_message = None -try: - from pyspark.sql.pandas.utils import require_minimum_pyarrow_version +from pyspark.sql import SparkSession +from pyspark.sql.types import ArrayType, DoubleType, UserDefinedType, Row +from pyspark.testing.utils import ( + ReusedPySparkTestCase, + PySparkErrorTestUtils, + have_pandas, + pandas_requirement_message, + have_pyarrow, + pyarrow_requirement_message, +) - require_minimum_pyarrow_version() -except ImportError as e: - # If Arrow version requirement is not satisfied, skip related tests. - pyarrow_requirement_message = str(e) test_not_compiled_message = None try: @@ -48,21 +42,6 @@ except Exception as e: test_not_compiled_message = str(e) -plotly_requirement_message = None -try: - import plotly -except ImportError as e: - plotly_requirement_message = str(e) -have_plotly = plotly_requirement_message is None - - -from pyspark.sql import SparkSession -from pyspark.sql.types import ArrayType, DoubleType, UserDefinedType, Row -from pyspark.testing.utils import ReusedPySparkTestCase, PySparkErrorTestUtils - - -have_pandas = pandas_requirement_message is None -have_pyarrow = pyarrow_requirement_message is None test_compiled = test_not_compiled_message is None diff --git a/python/pyspark/testing/utils.py b/python/pyspark/testing/utils.py index ca16628fc56f0..1dd15666382f6 100644 --- a/python/pyspark/testing/utils.py +++ b/python/pyspark/testing/utils.py @@ -82,6 +82,39 @@ def have_package(name: str) -> bool: have_plotly = have_package("plotly") plotly_requirement_message = None if have_plotly else "No module named 'plotly'" +have_matplotlib = have_package("matplotlib") +matplotlib_requirement_message = None if have_matplotlib else "No module named 'matplotlib'" + +have_tabulate = have_package("tabulate") +tabulate_requirement_message = None if have_tabulate else "No module named 'tabulate'" + +have_graphviz = have_package("graphviz") +graphviz_requirement_message = None if have_graphviz else "No module named 'graphviz'" + + +pandas_requirement_message = None +try: + from pyspark.sql.pandas.utils import require_minimum_pandas_version + + require_minimum_pandas_version() +except Exception as e: + # If Pandas version requirement is not satisfied, skip related tests. + pandas_requirement_message = str(e) + +have_pandas = pandas_requirement_message is None + + +pyarrow_requirement_message = None +try: + from pyspark.sql.pandas.utils import require_minimum_pyarrow_version + + require_minimum_pyarrow_version() +except Exception as e: + # If Arrow version requirement is not satisfied, skip related tests. + pyarrow_requirement_message = str(e) + +have_pyarrow = pyarrow_requirement_message is None + def read_int(b): return struct.unpack("!i", b)[0] From d3119fac0a09a2c6290762c9ba573378ccf30dfc Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 22 Nov 2024 12:05:22 +0900 Subject: [PATCH 48/58] [SPARK-50378][SS] Add custom metric for tracking spent for proc initial state in transformWithState ### What changes were proposed in this pull request? Add custom metric for tracking spent for proc initial state in transformWithState ### Why are the changes needed? Adds tracking for time spent in populating initial state ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit tests ``` [info] Run completed in 2 minutes, 38 seconds. [info] Total number of tests run: 22 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 22, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48913 from anishshri-db/task/SPARK-50378. Authored-by: Anish Shrigondekar Signed-off-by: Jungtaek Lim --- .../sql/execution/streaming/TransformWithStateExec.scala | 6 ++++++ .../streaming/TransformWithStateInitialStateSuite.scala | 8 ++++++++ 2 files changed, 14 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index 2b26d18019d12..107f98b09f858 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -410,6 +410,9 @@ case class TransformWithStateExec( // operator specific metrics override def customStatefulOperatorMetrics: Seq[StatefulOperatorCustomMetric] = { Seq( + // metrics around initial state + StatefulOperatorCustomSumMetric("initialStateProcessingTimeMs", + "Number of milliseconds taken to process all initial state"), // metrics around state variables StatefulOperatorCustomSumMetric("numValueStateVars", "Number of value state variables"), StatefulOperatorCustomSumMetric("numListStateVars", "Number of list state variables"), @@ -655,6 +658,8 @@ case class TransformWithStateExec( statefulProcessor.init(outputMode, timeMode) processorHandle.setHandleState(StatefulProcessorHandleState.INITIALIZED) + val initialStateProcTimeMs = longMetric("initialStateProcessingTimeMs") + val initialStateStartTimeNs = System.nanoTime // Check if is first batch // Only process initial states for first batch if (processorHandle.getQueryInfo().getBatchId == 0) { @@ -667,6 +672,7 @@ case class TransformWithStateExec( processInitialStateRows(keyRow.asInstanceOf[UnsafeRow], valueRowIter) } } + initialStateProcTimeMs += NANOSECONDS.toMillis(System.nanoTime - initialStateStartTimeNs) processDataWithPartition(childDataIterator, store, processorHandle) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala index 360656a76f350..806d2f19f6f5c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateInitialStateSuite.scala @@ -395,6 +395,10 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest AddData(inputData, InitInputRow("k2", "update", 40.0)), AddData(inputData, InitInputRow("non-exist", "getOption", -1.0)), CheckNewAnswer(("non-exist", "getOption", -1.0)), + Execute { q => + assert(q.lastProgress + .stateOperators(0).customMetrics.get("initialStateProcessingTimeMs") > 0) + }, AddData(inputData, InitInputRow("k1", "appendList", 37.0)), AddData(inputData, InitInputRow("k2", "appendList", 40.0)), AddData(inputData, InitInputRow("non-exist", "getList", -1.0)), @@ -514,6 +518,10 @@ class TransformWithStateInitialStateSuite extends StateStoreMetricsTest AdvanceManualClock(1 * 1000), // registered timer for "a" and "b" is 6000, first batch is processed at ts = 1000 CheckNewAnswer(("c", "1")), + Execute { q => + assert(q.lastProgress + .stateOperators(0).customMetrics.get("initialStateProcessingTimeMs") > 0) + }, AddData(inputData, "c"), AdvanceManualClock(6 * 1000), // ts = 7000, "a" expires From 985f98873d5127d1436f6e82de66c3348b106943 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 21 Nov 2024 21:00:16 -0800 Subject: [PATCH 49/58] [SPARK-48248][PYTHON][TESTS][FOLLOW-UP] Avoid duplicated test name for `test_infer_array_element_type_with_struct` ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/46548 that avoid duplicated test name. ### Why are the changes needed? Otherwise, one of both do not run. ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Manually ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48931 from HyukjinKwon/test-rename. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/tests/test_types.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 8a30608cd4087..dcc383b7add5a 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2310,7 +2310,7 @@ def test_collated_string(self): StringType("UTF8_LCASE"), ) - def test_infer_array_element_type_with_struct(self): + def test_infer_nested_array_element_type_with_struct(self): # SPARK-48248: Nested array to respect legacy conf of inferArrayTypeFromFirstElement with self.sql_conf( {"spark.sql.pyspark.legacy.inferArrayTypeFromFirstElement.enabled": True} From d9781d99ae877d624d7a1897fce0781ff3fc3323 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 21 Nov 2024 21:03:44 -0800 Subject: [PATCH 50/58] [SPARK-50389][SQL][TESTS] Remove`sql-tests/results/try_arithmetic.sql.out.java21` ### What changes were proposed in this pull request? There are 8 Java 21-related golden files in Spark: ``` ./sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/postgreSQL/float4.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/nonansi/try_aggregates.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/try_aggregates.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out.java21 ``` After SPARK-50313 was merged, the contents of `./sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21` and `./sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out` have become identical. We can verify by running `diff ./sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 ./sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out`. So this pr deletes this file. Actually, after SPARK-50313 the specialization for Java 21 has become `./sql/core/src/test/resources/sql-tests/results/nonansi/try_arithmetic.sql.out.java21`. ### Why are the changes needed? Delete an unnecessary golen file. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #48928 from LuciferYang/SPARK-50389. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../results/try_arithmetic.sql.out.java21 | 753 ------------------ 1 file changed, 753 deletions(-) delete mode 100644 sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 diff --git a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 b/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 deleted file mode 100644 index acf6e70a50dea..0000000000000 --- a/sql/core/src/test/resources/sql-tests/results/try_arithmetic.sql.out.java21 +++ /dev/null @@ -1,753 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- !query -SELECT try_add(1, 1) --- !query schema -struct --- !query output -2 - - --- !query -SELECT try_add(2147483647, 1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_add(2147483647, decimal(1)) --- !query schema -struct --- !query output -2147483648 - - --- !query -SELECT try_add(2147483647, "1") --- !query schema -struct --- !query output -2147483648 - - --- !query -SELECT try_add(-2147483648, -1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_add(9223372036854775807L, 1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_add(-9223372036854775808L, -1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_add(1, (2147483647 + 1)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 20, - "stopIndex" : 33, - "fragment" : "2147483647 + 1" - } ] -} - - --- !query -SELECT try_add(1L, (9223372036854775807L + 1L)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 21, - "stopIndex" : 45, - "fragment" : "9223372036854775807L + 1L" - } ] -} - - --- !query -SELECT try_add(1, 1.0 / 0.0) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 19, - "stopIndex" : 27, - "fragment" : "1.0 / 0.0" - } ] -} - - --- !query -SELECT try_add(date'2021-01-01', 1) --- !query schema -struct --- !query output -2021-01-02 - - --- !query -SELECT try_add(1, date'2021-01-01') --- !query schema -struct --- !query output -2021-01-02 - - --- !query -SELECT try_add(date'2021-01-01', interval 2 year) --- !query schema -struct --- !query output -2023-01-01 - - --- !query -SELECT try_add(date'2021-01-01', interval 2 second) --- !query schema -struct --- !query output -2021-01-01 00:00:02 - - --- !query -SELECT try_add(interval 2 year, date'2021-01-01') --- !query schema -struct --- !query output -2023-01-01 - - --- !query -SELECT try_add(interval 2 second, date'2021-01-01') --- !query schema -struct --- !query output -2021-01-01 00:00:02 - - --- !query -SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) --- !query schema -struct --- !query output -2023-01-01 00:00:00 - - --- !query -SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) --- !query schema -struct --- !query output -2021-01-01 00:00:02 - - --- !query -SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') --- !query schema -struct --- !query output -2023-01-01 00:00:00 - - --- !query -SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') --- !query schema -struct --- !query output -2021-01-01 00:00:02 - - --- !query -SELECT try_add(interval 2 year, interval 2 year) --- !query schema -struct --- !query output -4-0 - - --- !query -SELECT try_add(interval 2 second, interval 2 second) --- !query schema -struct --- !query output -0 00:00:04.000000000 - - --- !query -SELECT try_add(interval 2 year, interval 2 second) --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", - "sqlState" : "42K09", - "messageParameters" : { - "inputSql" : "\"INTERVAL '2' YEAR\"", - "inputType" : "\"INTERVAL YEAR\"", - "paramIndex" : "first", - "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", - "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 8, - "stopIndex" : 50, - "fragment" : "try_add(interval 2 year, interval 2 second)" - } ] -} - - --- !query -SELECT try_add(interval 2147483647 month, interval 2 month) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_add(interval 106751991 day, interval 3 day) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(1, 0.5) --- !query schema -struct --- !query output -2.000000 - - --- !query -SELECT try_divide(1, 0) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(0, 0) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(1, (2147483647 + 1)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 23, - "stopIndex" : 36, - "fragment" : "2147483647 + 1" - } ] -} - - --- !query -SELECT try_divide(1L, (9223372036854775807L + 1L)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 48, - "fragment" : "9223372036854775807L + 1L" - } ] -} - - --- !query -SELECT try_divide(1, 1.0 / 0.0) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 22, - "stopIndex" : 30, - "fragment" : "1.0 / 0.0" - } ] -} - - --- !query -SELECT try_divide(1, decimal(0)) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(1, "0") --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(interval 2 year, 2) --- !query schema -struct --- !query output -1-0 - - --- !query -SELECT try_divide(interval 2 second, 2) --- !query schema -struct --- !query output -0 00:00:01.000000000 - - --- !query -SELECT try_divide(interval 2 year, 0) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(interval 2 second, 0) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(interval 2147483647 month, 0.5) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_divide(interval 106751991 day, 0.5) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(1, 1) --- !query schema -struct --- !query output -0 - - --- !query -SELECT try_subtract(2147483647, -1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(2147483647, decimal(-1)) --- !query schema -struct --- !query output -2147483648 - - --- !query -SELECT try_subtract(2147483647, "-1") --- !query schema -struct --- !query output -2147483648 - - --- !query -SELECT try_subtract(-2147483648, 1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(9223372036854775807L, -1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(-9223372036854775808L, 1) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(1, (2147483647 + 1)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 25, - "stopIndex" : 38, - "fragment" : "2147483647 + 1" - } ] -} - - --- !query -SELECT try_subtract(1L, (9223372036854775807L + 1L)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 50, - "fragment" : "9223372036854775807L + 1L" - } ] -} - - --- !query -SELECT try_subtract(1, 1.0 / 0.0) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 32, - "fragment" : "1.0 / 0.0" - } ] -} - - --- !query -SELECT try_subtract(interval 2 year, interval 3 year) --- !query schema -struct --- !query output --1-0 - - --- !query -SELECT try_subtract(interval 3 second, interval 2 second) --- !query schema -struct --- !query output -0 00:00:01.000000000 - - --- !query -SELECT try_subtract(interval 2147483647 month, interval -2 month) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_subtract(interval 106751991 day, interval -3 day) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(2, 3) --- !query schema -struct --- !query output -6 - - --- !query -SELECT try_multiply(2147483647, -2) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(2147483647, decimal(-2)) --- !query schema -struct --- !query output --4294967294 - - --- !query -SELECT try_multiply(2147483647, "-2") --- !query schema -struct --- !query output --4294967294 - - --- !query -SELECT try_multiply(-2147483648, 2) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(9223372036854775807L, 2) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(-9223372036854775808L, -2) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(1, (2147483647 + 1)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "integer overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 25, - "stopIndex" : 38, - "fragment" : "2147483647 + 1" - } ] -} - - --- !query -SELECT try_multiply(1L, (9223372036854775807L + 1L)) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "ARITHMETIC_OVERFLOW", - "sqlState" : "22003", - "messageParameters" : { - "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", - "config" : "\"spark.sql.ansi.enabled\"", - "message" : "long overflow" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 26, - "stopIndex" : 50, - "fragment" : "9223372036854775807L + 1L" - } ] -} - - --- !query -SELECT try_multiply(1, 1.0 / 0.0) --- !query schema -struct<> --- !query output -org.apache.spark.SparkArithmeticException -{ - "errorClass" : "DIVIDE_BY_ZERO", - "sqlState" : "22012", - "messageParameters" : { - "config" : "\"spark.sql.ansi.enabled\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 24, - "stopIndex" : 32, - "fragment" : "1.0 / 0.0" - } ] -} - - --- !query -SELECT try_multiply(interval 2 year, 2) --- !query schema -struct --- !query output -4-0 - - --- !query -SELECT try_multiply(interval 2 second, 2) --- !query schema -struct --- !query output -0 00:00:04.000000000 - - --- !query -SELECT try_multiply(interval 2 year, 0) --- !query schema -struct --- !query output -0-0 - - --- !query -SELECT try_multiply(interval 2 second, 0) --- !query schema -struct --- !query output -0 00:00:00.000000000 - - --- !query -SELECT try_multiply(interval 2147483647 month, 2) --- !query schema -struct --- !query output -NULL - - --- !query -SELECT try_multiply(interval 106751991 day, 2) --- !query schema -struct --- !query output -NULL From 0fbd34c45b5818400f1993314acc571690daaf9a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 22 Nov 2024 13:20:37 +0800 Subject: [PATCH 51/58] [SPARK-50377][SQL] Allow to evaluate foldable RuntimeReplaceable ### What changes were proposed in this pull request? This is to fix a regression caused by https://github.com/apache/spark/pull/47143 . The problem is, in some places, we want to get a constant from a foldable expression before the query execution starts. https://github.com/apache/spark/pull/47143 brings two problems: 1. `UnaryPositive` is no longer a `UnaryExpression`, which means it's not foldable anymore even if its child is foldable. 2. `UnaryPositive` is no longer evaluable. `Lag` is such a place. It may evaluate the `inputOffset` parameter eagerly. `lag(..., +1)` no longer works after https://github.com/apache/spark/pull/47143 . Instead of fixing `Lag`, this PR makes two changes and hopefully we can avoid all similar problems: 1. Make `UnaryPositive` extend `UnaryExpression` again. We need follow-up PRs to check other `RuntimeReplaceable` expressions and see if they should extend `UnaryExpression` or `BinaryExpression`, etc. 2. Implement `RuntimeReplaceable#eval` so that we can evaluate folding `RuntimeReplaceable` eagerly when needed. ### Why are the changes needed? Fix the regression on `lag` and avoid similar issues in the future. ### Does this PR introduce _any_ user-facing change? No, the regression is not released yet. ### How was this patch tested? new test ### Was this patch authored or co-authored using generative AI tooling? no Closes #48912 from cloud-fan/replace. Lead-authored-by: Wenchen Fan Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/Expression.scala | 8 +++++-- .../sql/catalyst/expressions/arithmetic.scala | 10 +++----- .../sql-tests/analyzer-results/window.sql.out | 7 +++--- .../resources/sql-tests/inputs/window.sql | 1 + .../sql-tests/results/window.sql.out | 23 ++++++++++--------- 5 files changed, 26 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index f0f94f0881385..c454799852826 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -440,8 +440,12 @@ trait RuntimeReplaceable extends Expression { // are semantically equal. override lazy val canonicalized: Expression = replacement.canonicalized - final override def eval(input: InternalRow = null): Any = - throw QueryExecutionErrors.cannotEvaluateExpressionError(this) + final override def eval(input: InternalRow = null): Any = { + // For convenience, we allow to evaluate `RuntimeReplaceable` expressions, in case we need to + // get a constant from foldable expression before the query execution starts. + assert(input == null) + replacement.eval() + } final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = throw QueryExecutionErrors.cannotGenerateCodeForExpressionError(this) } 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 015240472cf5a..f9e8b6a17896f 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 @@ -115,8 +115,7 @@ case class UnaryMinus( since = "1.5.0", group = "math_funcs") case class UnaryPositive(child: Expression) - extends RuntimeReplaceable with ImplicitCastInputTypes { - override def nullIntolerant: Boolean = true + extends UnaryExpression with RuntimeReplaceable with ImplicitCastInputTypes { override def prettyName: String = "positive" @@ -128,11 +127,8 @@ case class UnaryPositive(child: Expression) override lazy val replacement: Expression = child - override protected def withNewChildrenInternal( - newChildren: IndexedSeq[Expression]): UnaryPositive = - copy(newChildren.head) - - override def children: Seq[Expression] = child :: Nil + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) } /** diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out index db49b1bfd39d8..a1a2b76006378 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/window.sql.out @@ -995,6 +995,7 @@ SELECT lag(v, 1) IGNORE NULLS OVER w lag_1, lag(v, 2) IGNORE NULLS OVER w lag_2, lag(v, 3) IGNORE NULLS OVER w lag_3, + lag(v, +3) IGNORE NULLS OVER w lag_plus_3, nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, @@ -1007,9 +1008,9 @@ WINDOW w AS (ORDER BY id) ORDER BY id -- !query analysis Sort [id#x ASC NULLS FIRST], true -+- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] - +- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, ... 7 more fields] - +- Window [lead(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lead_0#x, lead(v#x, 1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead_1#x, lead(v#x, 2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 2, 2)) AS lead_2#x, lead(v#x, 3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 3, 3)) AS lead_3#x, lag(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lag_0#x, lag(v#x, -1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -1)) AS lag_1#x, lag(v#x, -2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -2)) AS lag_2#x, lag(v#x, -3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -3, -3)) AS lag_3#x, nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x], [id#x ASC NULLS FIRST] ++- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, lag_plus_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x] + +- Project [content#x, id#x, v#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, lag_2#x, lag_3#x, lag_plus_3#x, nth_value_1#x, nth_value_2#x, nth_value_3#x, first_value#x, any_value#x, last_value#x, lead_0#x, lead_1#x, lead_2#x, lead_3#x, lag_0#x, lag_1#x, ... 9 more fields] + +- Window [lead(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lead_0#x, lead(v#x, 1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 1, 1)) AS lead_1#x, lead(v#x, 2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 2, 2)) AS lead_2#x, lead(v#x, 3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 3, 3)) AS lead_3#x, lag(v#x, 0, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, 0, 0)) AS lag_0#x, lag(v#x, -1, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -1, -1)) AS lag_1#x, lag(v#x, -2, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -2, -2)) AS lag_2#x, lag(v#x, -3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -3, -3)) AS lag_3#x, lag(v#x, -3, null) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RowFrame, -3, -3)) AS lag_plus_3#x, nth_value(v#x, 1, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_1#x, nth_value(v#x, 2, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_2#x, nth_value(v#x, 3, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS nth_value_3#x, first(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value#x, any_value(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS any_value#x, last(v#x, true) windowspecdefinition(id#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS last_value#x], [id#x ASC NULLS FIRST] +- Project [content#x, id#x, v#x] +- SubqueryAlias test_ignore_null +- View (`test_ignore_null`, [content#x, id#x, v#x]) diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index f3cbf6ef1ccb6..bec79247f9a61 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -327,6 +327,7 @@ SELECT lag(v, 1) IGNORE NULLS OVER w lag_1, lag(v, 2) IGNORE NULLS OVER w lag_2, lag(v, 3) IGNORE NULLS OVER w lag_3, + lag(v, +3) IGNORE NULLS OVER w lag_plus_3, nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 87381b64638b7..ce88fb57f8aa6 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1060,6 +1060,7 @@ SELECT lag(v, 1) IGNORE NULLS OVER w lag_1, lag(v, 2) IGNORE NULLS OVER w lag_2, lag(v, 3) IGNORE NULLS OVER w lag_3, + lag(v, +3) IGNORE NULLS OVER w lag_plus_3, nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, @@ -1071,17 +1072,17 @@ FROM WINDOW w AS (ORDER BY id) ORDER BY id -- !query schema -struct --- !query output -a 0 NULL NULL x y z NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -a 1 x x y z v x NULL NULL NULL x NULL NULL x x x -b 2 NULL NULL y z v NULL x NULL NULL x NULL NULL x x x -c 3 NULL NULL y z v NULL x NULL NULL x NULL NULL x x x -a 4 y y z v NULL y x NULL NULL x y NULL x x y -b 5 NULL NULL z v NULL NULL y x NULL x y NULL x x y -a 6 z z v NULL NULL z y x NULL x y z x x z -a 7 v v NULL NULL NULL v z y x x y z x x v -a 8 NULL NULL NULL NULL NULL NULL v z y x y z x x v +struct +-- !query output +a 0 NULL NULL x y z NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +a 1 x x y z v x NULL NULL NULL NULL x NULL NULL x x x +b 2 NULL NULL y z v NULL x NULL NULL NULL x NULL NULL x x x +c 3 NULL NULL y z v NULL x NULL NULL NULL x NULL NULL x x x +a 4 y y z v NULL y x NULL NULL NULL x y NULL x x y +b 5 NULL NULL z v NULL NULL y x NULL NULL x y NULL x x y +a 6 z z v NULL NULL z y x NULL NULL x y z x x z +a 7 v v NULL NULL NULL v z y x x x y z x x v +a 8 NULL NULL NULL NULL NULL NULL v z y y x y z x x v -- !query From ea222a3c72dd9ec5461583e7445347b4a7f54194 Mon Sep 17 00:00:00 2001 From: Daniel Tenedorio Date: Fri, 22 Nov 2024 13:28:57 +0800 Subject: [PATCH 52/58] [SPARK-49566][SQL] Add SQL pipe syntax for the EXTEND operator ### What changes were proposed in this pull request? This PR adds SQL pipe syntax support for the EXTEND operator. This operator preserves the existing input table and adds one or more new computed columns whose values are equal to evaluating the specified expressions. This is equivalent to `SELECT *, ` in the SQL compiler. It is provided as a convenience feature and some functionality overlap exists with lateral column aliases. For example: ``` CREATE TABLE t(x INT, y STRING) USING CSV; INSERT INTO t VALUES (0, 'abc'), (1, 'def'); TABLE t |> EXTEND x + LENGTH(y) AS z; +----+-----+-----+ | x | y | z | +----+-----+-----+ | 0 | abc | 3 | | 1 | def | 4 | +----+-----+-----+ ``` Like the `|> SELECT` operator, aggregate functions are not allowed in these expressions. During the course of developing reasonable error messages for this, I found that the SQL pipe syntax research paper also specified that the `|> AGGREGATE` operator should require that each non-grouping expression contains at least one aggregate function; I added a check and reasonable error message for this case as well. ### Why are the changes needed? The SQL pipe operator syntax will let users compose queries in a more flexible fashion. ### Does this PR introduce _any_ user-facing change? Yes, see above. ### How was this patch tested? This PR adds a few unit test cases, but mostly relies on golden file test coverage. I did this to make sure the answers are correct as this feature is implemented and also so we can look at the analyzer output plans to ensure they look right as well. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48854 from dtenedor/pipe-syntax-projections. Authored-by: Daniel Tenedorio Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 10 +- docs/sql-ref-ansi-compliance.md | 1 + .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../sql/catalyst/parser/SqlBaseParser.g4 | 3 + .../catalyst/expressions/pipeOperators.scala | 48 +-- .../sql/catalyst/parser/AstBuilder.scala | 38 ++- .../sql/errors/QueryCompilationErrors.scala | 13 +- .../analyzer-results/pipe-operators.sql.out | 283 ++++++++++++++---- .../sql-tests/inputs/pipe-operators.sql | 84 +++++- .../results/keywords-enforced.sql.out | 1 + .../sql-tests/results/keywords.sql.out | 1 + .../results/nonansi/keywords.sql.out | 1 + .../sql-tests/results/pipe-operators.sql.out | 251 +++++++++++++--- .../sql/execution/SparkSqlParserSuite.scala | 4 +- .../ThriftServerWithSparkContextSuite.scala | 2 +- 15 files changed, 618 insertions(+), 123 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 38b1656ac05c1..94513cca1023f 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3995,9 +3995,15 @@ ], "sqlState" : "42K03" }, - "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION" : { + "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION" : { "message" : [ - "Aggregate function is not allowed when using the pipe operator |> SELECT clause; please use the pipe operator |> AGGREGATE clause instead" + "Non-grouping expression is provided as an argument to the |> AGGREGATE pipe operator but does not contain any aggregate function; please update it to include an aggregate function and then retry the query again." + ], + "sqlState" : "0A000" + }, + "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION" : { + "message" : [ + "Aggregate function is not allowed when using the pipe operator |> clause; please use the pipe operator |> AGGREGATE clause instead." ], "sqlState" : "0A000" }, diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index e05c33509400d..7af54850f5da7 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -514,6 +514,7 @@ Below is a list of all the keywords in Spark SQL. |EXISTS|non-reserved|non-reserved|reserved| |EXPLAIN|non-reserved|non-reserved|non-reserved| |EXPORT|non-reserved|non-reserved|non-reserved| +|EXTEND|non-reserved|non-reserved|non-reserved| |EXTENDED|non-reserved|non-reserved|non-reserved| |EXTERNAL|non-reserved|non-reserved|reserved| |EXTRACT|non-reserved|non-reserved|reserved| diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index 085e723d02bc0..eeebe89de8ff1 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -228,6 +228,7 @@ EXCLUDE: 'EXCLUDE'; EXISTS: 'EXISTS'; EXPLAIN: 'EXPLAIN'; EXPORT: 'EXPORT'; +EXTEND: 'EXTEND'; EXTENDED: 'EXTENDED'; EXTERNAL: 'EXTERNAL'; EXTRACT: 'EXTRACT'; diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 55a4b85ecb6bf..cdee8c906054d 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -1503,6 +1503,7 @@ version operatorPipeRightSide : selectClause windowClause? + | EXTEND extendList=namedExpressionSeq // Note that the WINDOW clause is not allowed in the WHERE pipe operator, but we add it here in // the grammar simply for purposes of catching this invalid syntax and throwing a specific // dedicated error message. @@ -1617,6 +1618,7 @@ ansiNonReserved | EXISTS | EXPLAIN | EXPORT + | EXTEND | EXTENDED | EXTERNAL | EXTRACT @@ -1963,6 +1965,7 @@ nonReserved | EXISTS | EXPLAIN | EXPORT + | EXTEND | EXTENDED | EXTERNAL | EXTRACT diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala index a0f2198212689..1b5ee54729136 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/pipeOperators.scala @@ -18,41 +18,55 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction -import org.apache.spark.sql.catalyst.trees.TreePattern.{PIPE_OPERATOR_SELECT, RUNTIME_REPLACEABLE, TreePattern} import org.apache.spark.sql.errors.QueryCompilationErrors /** - * Represents a SELECT clause when used with the |> SQL pipe operator. - * We use this to make sure that no aggregate functions exist in the SELECT expressions. + * Represents an expression when used with a SQL pipe operator. + * We use this to check invariants about whether aggregate functions may exist in these expressions. + * @param child The child expression. + * @param isAggregate Whether the pipe operator is |> AGGREGATE. + * If true, the child expression must contain at least one aggregate function. + * If false, the child expression must not contain any aggregate functions. + * @param clause The clause of the pipe operator. This is used to generate error messages. */ -case class PipeSelect(child: Expression) +case class PipeExpression(child: Expression, isAggregate: Boolean, clause: String) extends UnaryExpression with RuntimeReplaceable { - final override val nodePatterns: Seq[TreePattern] = Seq(PIPE_OPERATOR_SELECT, RUNTIME_REPLACEABLE) - override def withNewChildInternal(newChild: Expression): Expression = PipeSelect(newChild) + override def withNewChildInternal(newChild: Expression): Expression = + PipeExpression(newChild, isAggregate, clause) override lazy val replacement: Expression = { - def visit(e: Expression): Unit = e match { - case a: AggregateFunction => - // If we used the pipe operator |> SELECT clause to specify an aggregate function, this is - // invalid; return an error message instructing the user to use the pipe operator - // |> AGGREGATE clause for this purpose instead. - throw QueryCompilationErrors.pipeOperatorSelectContainsAggregateFunction(a) - case _: WindowExpression => - // Window functions are allowed in pipe SELECT operators, so do not traverse into children. - case _ => - e.children.foreach(visit) + val firstAggregateFunction: Option[AggregateFunction] = findFirstAggregate(child) + if (isAggregate && firstAggregateFunction.isEmpty) { + throw QueryCompilationErrors.pipeOperatorAggregateExpressionContainsNoAggregateFunction(child) + } else if (!isAggregate) { + firstAggregateFunction.foreach { a => + throw QueryCompilationErrors.pipeOperatorContainsAggregateFunction(a, clause) + } } - visit(child) child } + + /** Returns the first aggregate function in the given expression, or None if not found. */ + private def findFirstAggregate(e: Expression): Option[AggregateFunction] = e match { + case a: AggregateFunction => + Some(a) + case _: WindowExpression => + // Window functions are allowed in these pipe operators, so do not traverse into children. + None + case _ => + e.children.flatMap(findFirstAggregate).headOption + } } object PipeOperators { // These are definitions of query result clauses that can be used with the pipe operator. + val aggregateClause = "AGGREGATE" val clusterByClause = "CLUSTER BY" val distributeByClause = "DISTRIBUTE BY" + val extendClause = "EXTEND" val limitClause = "LIMIT" val offsetClause = "OFFSET" val orderByClause = "ORDER BY" + val selectClause = "SELECT" val sortByClause = "SORT BY" val sortByDistributeByClause = "SORT BY ... DISTRIBUTE BY ..." val windowClause = "WINDOW" 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 1604a0b5aba19..08a8cf6bab87a 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 @@ -1236,7 +1236,7 @@ class AstBuilder extends DataTypeAstBuilder * Add a regular (SELECT) query specification to a logical plan. The query specification * is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT), * aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place. - * If 'isPipeOperatorSelect' is true, wraps each projected expression with a [[PipeSelect]] + * If 'isPipeOperatorSelect' is true, wraps each projected expression with a [[PipeExpression]] * expression for future validation of the expressions during analysis. * * Note that query hints are ignored (both by the parser and the builder). @@ -1293,11 +1293,12 @@ class AstBuilder extends DataTypeAstBuilder def createProject() = if (namedExpressions.nonEmpty) { val newProjectList: Seq[NamedExpression] = if (isPipeOperatorSelect) { - // If this is a pipe operator |> SELECT clause, add a [[PipeSelect]] expression wrapping + // If this is a pipe operator |> SELECT clause, add a [[PipeExpression]] wrapping // each alias in the project list, so the analyzer can check invariants later. namedExpressions.map { case a: Alias => - a.withNewChildren(Seq(PipeSelect(a.child))) + a.withNewChildren(Seq( + PipeExpression(a.child, isAggregate = false, PipeOperators.selectClause))) .asInstanceOf[NamedExpression] case other => other @@ -5933,6 +5934,24 @@ class AstBuilder extends DataTypeAstBuilder windowClause = ctx.windowClause, relation = left, isPipeOperatorSelect = true) + }.getOrElse(Option(ctx.EXTEND).map { _ => + // Visit each expression in the EXTEND operator, and add a PipeExpression expression on top of + // it to generate clear error messages if the expression contains any aggregate functions + // (this is not allowed in the EXTEND operator). + val extendExpressions: Seq[NamedExpression] = + Option(ctx.extendList).map { n: NamedExpressionSeqContext => + visitNamedExpressionSeq(n).map { + case (a: Alias, _) => + a.copy( + child = PipeExpression(a.child, isAggregate = false, PipeOperators.extendClause))( + a.exprId, a.qualifier, a.explicitMetadata, a.nonInheritableMetadataKeys) + case (e: Expression, aliasFunc) => + UnresolvedAlias( + PipeExpression(e, isAggregate = false, PipeOperators.extendClause), aliasFunc) + } + }.get + val projectList: Seq[NamedExpression] = Seq(UnresolvedStar(None)) ++ extendExpressions + Project(projectList, left) }.getOrElse(Option(ctx.whereClause).map { c => if (ctx.windowClause() != null) { throw QueryParsingErrors.windowClauseInPipeOperatorWhereClauseNotAllowedError(ctx) @@ -5959,7 +5978,7 @@ class AstBuilder extends DataTypeAstBuilder withQueryResultClauses(c, withSubqueryAlias(), forPipeOperators = true) }.getOrElse( visitOperatorPipeAggregate(ctx, left) - )))))))) + ))))))))) } private def visitOperatorPipeAggregate( @@ -5970,11 +5989,18 @@ class AstBuilder extends DataTypeAstBuilder "The AGGREGATE clause requires a list of aggregate expressions " + "or a list of grouping expressions, or both", ctx) } + // Visit each aggregate expression, and add a PipeAggregate expression on top of it to generate + // clear error messages if the expression does not contain at least one aggregate function. val aggregateExpressions: Seq[NamedExpression] = Option(ctx.namedExpressionSeq()).map { n: NamedExpressionSeqContext => visitNamedExpressionSeq(n).map { - case (e: NamedExpression, _) => e - case (e: Expression, aliasFunc) => UnresolvedAlias(e, aliasFunc) + case (a: Alias, _) => + a.copy(child = + PipeExpression(a.child, isAggregate = true, PipeOperators.aggregateClause))( + a.exprId, a.qualifier, a.explicitMetadata, a.nonInheritableMetadataKeys) + case (e: Expression, aliasFunc) => + UnresolvedAlias( + PipeExpression(e, isAggregate = true, PipeOperators.aggregateClause), aliasFunc) } }.getOrElse(Seq.empty) Option(ctx.aggregationClause()).map { c: AggregationClauseContext => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index b628412929e37..03471ae8a3da5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4135,14 +4135,23 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat ) } - def pipeOperatorSelectContainsAggregateFunction(expr: Expression): Throwable = { + def pipeOperatorAggregateExpressionContainsNoAggregateFunction(expr: Expression): Throwable = { new AnalysisException( - errorClass = "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + errorClass = "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", messageParameters = Map( "expr" -> expr.toString), origin = expr.origin) } + def pipeOperatorContainsAggregateFunction(expr: Expression, clause: String): Throwable = { + new AnalysisException( + errorClass = "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + messageParameters = Map( + "expr" -> expr.toString, + "clause" -> clause), + origin = expr.origin) + } + def inlineTableContainsScalarSubquery(inlineTable: LogicalPlan): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.SCALAR_SUBQUERY_IN_VALUES", diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out index 2e38ed137b415..bc5b642e3da09 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/pipe-operators.sql.out @@ -269,7 +269,7 @@ CreateViewCommand `windowTestData`, select * from values table t |> select 1 as x -- !query analysis -Project [pipeselect(1) AS x#x] +Project [pipeexpression(1, false, SELECT) AS x#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -288,7 +288,7 @@ table t |> select x, y |> select x + length(y) as z -- !query analysis -Project [pipeselect((x#x + length(y#x))) AS z#x] +Project [pipeexpression((x#x + length(y#x)), false, SELECT) AS z#x] +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -298,7 +298,7 @@ Project [pipeselect((x#x + length(y#x))) AS z#x] values (0), (1) tab(col) |> select col * 2 as result -- !query analysis -Project [pipeselect((col#x * 2)) AS result#x] +Project [pipeexpression((col#x * 2), false, SELECT) AS result#x] +- SubqueryAlias tab +- LocalRelation [col#x] @@ -307,7 +307,7 @@ Project [pipeselect((col#x * 2)) AS result#x] (select * from t union all select * from t) |> select x + length(y) as result -- !query analysis -Project [pipeselect((x#x + length(y#x))) AS result#x] +Project [pipeexpression((x#x + length(y#x)), false, SELECT) AS result#x] +- Union false, false :- Project [x#x, y#x] : +- SubqueryAlias spark_catalog.default.t @@ -358,7 +358,7 @@ Project [col#x.i1 AS i1#x] table t |> select (select a from other where x = a limit 1) as result -- !query analysis -Project [pipeselect(scalar-subquery#x [x#x]) AS result#x] +Project [pipeexpression(scalar-subquery#x [x#x], false, SELECT) AS result#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Project [a#x] @@ -383,7 +383,7 @@ Project [scalar-subquery#x [] AS result#x] table t |> select (select any_value(a) from other where x = a limit 1) as result -- !query analysis -Project [pipeselect(scalar-subquery#x [x#x]) AS result#x] +Project [pipeexpression(scalar-subquery#x [x#x], false, SELECT) AS result#x] : +- GlobalLimit 1 : +- LocalLimit 1 : +- Aggregate [any_value(a#x, false) AS any_value(a)#x] @@ -398,8 +398,8 @@ Project [pipeselect(scalar-subquery#x [x#x]) AS result#x] table t |> select x + length(x) as z, z + 1 as plus_one -- !query analysis -Project [z#x, pipeselect((z#x + 1)) AS plus_one#x] -+- Project [x#x, y#x, pipeselect((x#x + length(cast(x#x as string)))) AS z#x] +Project [z#x, pipeexpression((z#x + 1), false, SELECT) AS plus_one#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(cast(x#x as string))), false, SELECT) AS z#x] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -409,7 +409,7 @@ table t |> select first_value(x) over (partition by y) as result -- !query analysis Project [result#x] -+- Project [x#x, y#x, _we0#x, pipeselect(_we0#x) AS result#x] ++- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, SELECT) AS result#x] +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] +- Project [x#x, y#x] +- SubqueryAlias spark_catalog.default.t @@ -426,7 +426,7 @@ select 1 x, 2 y, 3 z -- !query analysis Project [a2#x] +- Project [(1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, x#x, a2#x] - +- Project [x#x, y#x, _w1#x, z#x, _we0#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, _we2#x, (cast(1 as bigint) + _we0#xL) AS (1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, pipeselect(_we2#x) AS a2#x] + +- Project [x#x, y#x, _w1#x, z#x, _we0#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, _we2#x, (cast(1 as bigint) + _we0#xL) AS (1 + sum(x) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))#xL, avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x, pipeexpression(_we2#x, false, SELECT) AS a2#x] +- Window [avg(_w1#x) windowspecdefinition(y#x, z#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS _we2#x], [y#x], [z#x ASC NULLS FIRST] +- Window [sum(x#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#xL, avg(y#x) windowspecdefinition(specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg(y) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)#x] +- Project [x#x, y#x, (x#x + 1) AS _w1#x, z#x] @@ -513,9 +513,10 @@ table t -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(x#x)" }, "queryContext" : [ { @@ -534,9 +535,10 @@ table t -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(x#x)" }, "queryContext" : [ { @@ -549,6 +551,186 @@ org.apache.spark.sql.AnalysisException } +-- !query +table t +|> extend 1 as z +-- !query analysis +Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS z#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 +-- !query analysis +Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS pipeexpression(1)#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend x as z +-- !query analysis +Project [x#x, y#x, pipeexpression(x#x, false, EXTEND) AS z#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend x + length(y) as z +-- !query analysis +Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend x + length(y) as z, x + 1 as zz +-- !query analysis +Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x, pipeexpression((x#x + 1), false, EXTEND) AS zz#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend x + length(y) as z +|> extend z + 1 as zz +-- !query analysis +Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, EXTEND) AS zz#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +select col from st +|> extend col.i1 as z +-- !query analysis +Project [col#x, pipeexpression(col#x.i1, false, EXTEND) AS z#x] ++- Project [col#x] + +- SubqueryAlias spark_catalog.default.st + +- Relation spark_catalog.default.st[x#x,col#x] parquet + + +-- !query +table t +|> extend (select a from other where x = a limit 1) as z +-- !query analysis +Project [x#x, y#x, pipeexpression(scalar-subquery#x [x#x], false, EXTEND) AS z#x] +: +- GlobalLimit 1 +: +- LocalLimit 1 +: +- Project [a#x] +: +- Filter (outer(x#x) = a#x) +: +- SubqueryAlias spark_catalog.default.other +: +- Relation spark_catalog.default.other[a#x,b#x] json ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> where exists ( + table other + |> extend t.x + |> select * except (a, b)) +-- !query analysis +Filter exists#x [x#x] +: +- Project [pipeexpression(outer(spark_catalog.default.t.x))#x] +: +- Project [a#x, b#x, pipeexpression(outer(x#x), false, EXTEND) AS pipeexpression(outer(spark_catalog.default.t.x))#x] +: +- SubqueryAlias spark_catalog.default.other +: +- Relation spark_catalog.default.other[a#x,b#x] json ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend 1 as x +-- !query analysis +Project [x#x, y#x, pipeexpression(1, false, EXTEND) AS x#x] ++- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend first_value(x) over (partition by y) as result +-- !query analysis +Project [x#x, y#x, result#x] ++- Project [x#x, y#x, _we0#x, pipeexpression(_we0#x, false, EXTEND) AS result#x] + +- Window [first_value(x#x, false) windowspecdefinition(y#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#x], [y#x] + +- Project [x#x, y#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend x + length(y) as z, z + 1 as plus_one +-- !query analysis +Project [x#x, y#x, z#x, pipeexpression((z#x + 1), false, EXTEND) AS plus_one#x] ++- Project [x#x, y#x, pipeexpression((x#x + length(y#x)), false, EXTEND) AS z#x] + +- SubqueryAlias spark_catalog.default.t + +- Relation spark_catalog.default.t[x#x,y#x] csv + + +-- !query +table t +|> extend sum(x) as z +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "EXTEND", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 24, + "fragment" : "sum(x)" + } ] +} + + +-- !query +table t +|> extend distinct x as z +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +table t +|> extend * +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_USAGE_OF_STAR_OR_REGEX", + "sqlState" : "42000", + "messageParameters" : { + "elem" : "'*'", + "prettyName" : "expression `pipeexpression`" + } +} + + -- !query table t |> where true @@ -822,7 +1004,7 @@ table courseSales Project [c#x, __pivot_sum(e) AS s AS `sum(e) AS s`#x[0] AS firstYear_s#xL, __pivot_avg(e) AS a AS `avg(e) AS a`#x[0] AS firstYear_a#x, __pivot_sum(e) AS s AS `sum(e) AS s`#x[1] AS secondYear_s#xL, __pivot_avg(e) AS a AS `avg(e) AS a`#x[1] AS secondYear_a#x] +- Aggregate [c#x], [c#x, pivotfirst(y#x, sum(e) AS s#xL, 2012, 2013, 0, 0) AS __pivot_sum(e) AS s AS `sum(e) AS s`#x, pivotfirst(y#x, avg(e) AS a#x, 2012, 2013, 0, 0) AS __pivot_avg(e) AS a AS `avg(e) AS a`#x] +- Aggregate [c#x, y#x], [c#x, y#x, sum(e#x) AS sum(e) AS s#xL, avg(e#x) AS avg(e) AS a#x] - +- Project [pipeselect(year#x) AS y#x, pipeselect(course#x) AS c#x, pipeselect(earnings#x) AS e#x] + +- Project [pipeexpression(year#x, false, SELECT) AS y#x, pipeexpression(course#x, false, SELECT) AS c#x, pipeexpression(earnings#x, false, SELECT) AS e#x] +- SubqueryAlias coursesales +- View (`courseSales`, [course#x, year#x, earnings#x]) +- Project [cast(course#x as string) AS course#x, cast(year#x as int) AS year#x, cast(earnings#x as int) AS earnings#x] @@ -2300,7 +2482,7 @@ org.apache.spark.sql.catalyst.parser.ParseException table other |> aggregate sum(b) as result group by a -- !query analysis -Aggregate [a#x], [a#x, sum(b#x) AS result#xL] +Aggregate [a#x], [a#x, pipeexpression(sum(b#x), true, AGGREGATE) AS result#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -2311,7 +2493,7 @@ table other |> select result -- !query analysis Project [result#xL] -+- Aggregate [a#x], [a#x, sum(b#x) AS result#xL] ++- Aggregate [a#x], [a#x, pipeexpression(sum(b#x), true, AGGREGATE) AS result#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -2322,7 +2504,7 @@ table other |> select gkey -- !query analysis Project [gkey#x] -+- Aggregate [(a#x + 1)], [(a#x + 1) AS gkey#x, sum(b#x) AS sum(b)#xL] ++- Aggregate [(a#x + 1)], [(a#x + 1) AS gkey#x, pipeexpression(sum(b#x), true, AGGREGATE) AS pipeexpression(sum(b))#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json @@ -2349,7 +2531,7 @@ Aggregate [1, 2], [1 AS 1#x, 2 AS 2#x] table t |> aggregate sum(x) -- !query analysis -Aggregate [sum(x#x) AS sum(x)#xL] +Aggregate [pipeexpression(sum(x#x), true, AGGREGATE) AS pipeexpression(sum(x))#xL] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -2358,7 +2540,7 @@ Aggregate [sum(x#x) AS sum(x)#xL] table t |> aggregate sum(x) + 1 as result_plus_one -- !query analysis -Aggregate [(sum(x#x) + cast(1 as bigint)) AS result_plus_one#xL] +Aggregate [pipeexpression((sum(x#x) + cast(1 as bigint)), true, AGGREGATE) AS result_plus_one#xL] +- SubqueryAlias spark_catalog.default.t +- Relation spark_catalog.default.t[x#x,y#x] csv @@ -2408,9 +2590,9 @@ select 1 x, 2 y, 3 z |> aggregate avg(z) z group by x |> aggregate count(distinct z) c -- !query analysis -Aggregate [count(distinct z#x) AS c#xL] -+- Aggregate [x#x], [x#x, avg(z#xL) AS z#x] - +- Aggregate [x#x, y#x], [x#x, y#x, sum(z#x) AS z#xL] +Aggregate [pipeexpression(count(distinct z#x), true, AGGREGATE) AS c#xL] ++- Aggregate [x#x], [x#x, pipeexpression(avg(z#xL), true, AGGREGATE) AS z#x] + +- Aggregate [x#x, y#x], [x#x, y#x, pipeexpression(sum(z#x), true, AGGREGATE) AS z#xL] +- Project [1 AS x#x, 2 AS y#x, 3 AS z#x] +- OneRowRelation @@ -2421,27 +2603,39 @@ select 1 x, 3 z |> select x -- !query analysis Project [x#x] -+- Aggregate [x#x, z#x, x#x], [x#x, z#x, x#x, count(1) AS count(1)#xL] ++- Aggregate [x#x, z#x, x#x], [x#x, z#x, x#x, pipeexpression(count(1), true, AGGREGATE) AS pipeexpression(count(1))#xL] +- Project [1 AS x#x, 3 AS z#x] +- OneRowRelation -- !query table other -|> aggregate a group by a +|> aggregate a + count(b) group by a -- !query analysis -Aggregate [a#x], [a#x, a#x] +Aggregate [a#x], [a#x, pipeexpression((cast(a#x as bigint) + count(b#x)), true, AGGREGATE) AS pipeexpression((a + count(b)))#xL] +- SubqueryAlias spark_catalog.default.other +- Relation spark_catalog.default.other[a#x,b#x] json -- !query table other -|> aggregate a + count(b) group by a +|> aggregate a group by a -- !query analysis -Aggregate [a#x], [a#x, (cast(a#x as bigint) + count(b#x)) AS (a + count(b))#xL] -+- SubqueryAlias spark_catalog.default.other - +- Relation spark_catalog.default.other[a#x,b#x] json +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} -- !query @@ -2596,16 +2790,19 @@ org.apache.spark.sql.catalyst.parser.ParseException table other |> aggregate a -- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "MISSING_GROUP_BY", - "sqlState" : "42803", + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 26, "stopIndex" : 26, - "fragment" : "table other\n|> aggregate a" + "fragment" : "a" } ] } @@ -2616,9 +2813,10 @@ table other -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(a#x)" }, "queryContext" : [ { @@ -2733,21 +2931,6 @@ org.apache.spark.sql.AnalysisException } --- !query -table other -|> aggregate b group by a --- !query analysis -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_AGGREGATION", - "sqlState" : "42803", - "messageParameters" : { - "expression" : "\"b\"", - "expressionAnyValue" : "\"any_value(b)\"" - } -} - - -- !query table windowTestData |> select cate, sum(val) over w @@ -2808,7 +2991,7 @@ Project [cate#x, val#x, sum_val#xL, first_value(cate) OVER (ORDER BY val ASC NUL +- Window [first_value(cate#x, false) windowspecdefinition(val#x ASC NULLS FIRST, specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS first_value(cate) OVER (ORDER BY val ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)#x], [val#x ASC NULLS FIRST] +- Project [cate#x, val#x, sum_val#xL] +- Project [cate#x, val#x, sum_val#xL] - +- Project [cate#x, val#x, _we0#xL, pipeselect(_we0#xL) AS sum_val#xL] + +- Project [cate#x, val#x, _we0#xL, pipeexpression(_we0#xL, false, SELECT) AS sum_val#xL] +- Window [sum(val#x) windowspecdefinition(cate#x, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#xL], [cate#x] +- Project [cate#x, val#x] +- SubqueryAlias windowtestdata diff --git a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql index 6261bc93b1856..b9224db129ea4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pipe-operators.sql @@ -171,6 +171,76 @@ table t table t |> select y, length(y) + sum(x) as result; +-- EXTEND operators: positive tests. +------------------------------------ + +-- Extending with a constant. +table t +|> extend 1 as z; + +-- Extending without an explicit alias. +table t +|> extend 1; + +-- Extending with an attribute. +table t +|> extend x as z; + +-- Extending with an expression. +table t +|> extend x + length(y) as z; + +-- Extending two times. +table t +|> extend x + length(y) as z, x + 1 as zz; + +-- Extending two times in sequence. +table t +|> extend x + length(y) as z +|> extend z + 1 as zz; + +-- Extending with a struct field. +select col from st +|> extend col.i1 as z; + +-- Extending with a subquery. +table t +|> extend (select a from other where x = a limit 1) as z; + +-- Extending with a correlated reference. +table t +|> where exists ( + table other + |> extend t.x + |> select * except (a, b)); + +-- Extending with a column name that already exists in the input relation. +table t +|> extend 1 as x; + +-- Window functions are allowed in the pipe operator EXTEND list. +table t +|> extend first_value(x) over (partition by y) as result; + +-- Lateral column aliases in the pipe operator EXTEND list. +table t +|> extend x + length(y) as z, z + 1 as plus_one; + +-- EXTEND operators: negative tests. +------------------------------------ + +-- Aggregations are not allowed. +table t +|> extend sum(x) as z; + +-- DISTINCT is not supported. +table t +|> extend distinct x as z; + +-- EXTEND * is not supported. +table t +|> extend *; + -- WHERE operators: positive tests. ----------------------------------- @@ -738,11 +808,6 @@ select 1 x, 3 z |> aggregate count(*) group by x, z, x |> select x; --- Grouping expressions are allowed in the aggregate functions list if they appear separately in the --- GROUP BY clause. -table other -|> aggregate a group by a; - -- Aggregate expressions may contain a mix of aggregate functions and grouping expressions. table other |> aggregate a + count(b) group by a; @@ -750,6 +815,10 @@ table other -- Aggregation operators: negative tests. ----------------------------------------- +-- All aggregate expressions must contain at least one aggregate function. +table other +|> aggregate a group by a; + -- GROUP BY ALL is not currently supported. select 3 as x, 4 as y |> aggregate group by all; @@ -815,11 +884,6 @@ select 1 x, 2 y, 3 z |> where c = 1 |> where x = 1; --- Aggregate expressions may not contain references to columns or expressions not otherwise listed --- in the GROUP BY clause. -table other -|> aggregate b group by a; - -- WINDOW operators (within SELECT): positive tests. --------------------------------------------------- diff --git a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out index b2331ec4ab804..7d96a3e98c832 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out @@ -115,6 +115,7 @@ EXECUTE true EXISTS false EXPLAIN false EXPORT false +EXTEND false EXTENDED false EXTERNAL false EXTRACT false diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out index a885525028623..6cbfe519a76f6 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out @@ -115,6 +115,7 @@ EXECUTE false EXISTS false EXPLAIN false EXPORT false +EXTEND false EXTENDED false EXTERNAL false EXTRACT false diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out index a885525028623..6cbfe519a76f6 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out @@ -115,6 +115,7 @@ EXECUTE false EXISTS false EXPLAIN false EXPORT false +EXTEND false EXTENDED false EXTERNAL false EXTRACT false diff --git a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out index a365e759b7c14..53aabce4d5abc 100644 --- a/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pipe-operators.sql.out @@ -457,9 +457,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(x#x)" }, "queryContext" : [ { @@ -480,9 +481,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(x#x)" }, "queryContext" : [ { @@ -495,6 +497,187 @@ org.apache.spark.sql.AnalysisException } +-- !query +table t +|> extend 1 as z +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend 1 +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend x as z +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +table t +|> extend x + length(y) as z +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend x + length(y) as z, x + 1 as zz +-- !query schema +struct +-- !query output +0 abc 3 1 +1 def 4 2 + + +-- !query +table t +|> extend x + length(y) as z +|> extend z + 1 as zz +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +select col from st +|> extend col.i1 as z +-- !query schema +struct,z:int> +-- !query output +{"i1":2,"i2":3} 2 + + +-- !query +table t +|> extend (select a from other where x = a limit 1) as z +-- !query schema +struct +-- !query output +0 abc NULL +1 def 1 + + +-- !query +table t +|> where exists ( + table other + |> extend t.x + |> select * except (a, b)) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> extend 1 as x +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend first_value(x) over (partition by y) as result +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +table t +|> extend x + length(y) as z, z + 1 as plus_one +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +table t +|> extend sum(x) as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "EXTEND", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 24, + "fragment" : "sum(x)" + } ] +} + + +-- !query +table t +|> extend distinct x as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +table t +|> extend * +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_USAGE_OF_STAR_OR_REGEX", + "sqlState" : "42000", + "messageParameters" : { + "elem" : "'*'", + "prettyName" : "expression `pipeexpression`" + } +} + + -- !query table t |> where true @@ -2005,7 +2188,7 @@ struct<1:int,2:int> table t |> aggregate sum(x) -- !query schema -struct +struct -- !query output 1 @@ -2079,22 +2262,35 @@ struct -- !query table other -|> aggregate a group by a +|> aggregate a + count(b) group by a -- !query schema -struct +struct -- !query output -1 1 -2 2 +1 3 +2 3 -- !query table other -|> aggregate a + count(b) group by a +|> aggregate a group by a -- !query schema -struct +struct<> -- !query output -1 3 -2 3 +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} -- !query @@ -2265,16 +2461,19 @@ table other -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException +org.apache.spark.sql.AnalysisException { - "errorClass" : "MISSING_GROUP_BY", - "sqlState" : "42803", + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 1, + "startIndex" : 26, "stopIndex" : 26, - "fragment" : "table other\n|> aggregate a" + "fragment" : "a" } ] } @@ -2287,9 +2486,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "PIPE_OPERATOR_SELECT_CONTAINS_AGGREGATE_FUNCTION", + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", "sqlState" : "0A000", "messageParameters" : { + "clause" : "SELECT", "expr" : "sum(a#x)" }, "queryContext" : [ { @@ -2414,23 +2614,6 @@ org.apache.spark.sql.AnalysisException } --- !query -table other -|> aggregate b group by a --- !query schema -struct<> --- !query output -org.apache.spark.sql.catalyst.ExtendedAnalysisException -{ - "errorClass" : "MISSING_AGGREGATION", - "sqlState" : "42803", - "messageParameters" : { - "expression" : "\"b\"", - "expressionAnyValue" : "\"any_value(b)\"" - } -} - - -- !query table windowTestData |> select cate, sum(val) over w diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 357fd8beb961a..03d6eb1a50209 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -891,13 +891,15 @@ class SparkSqlParserSuite extends AnalysisTest with SharedSparkSession { // inline table. def check(query: String, patterns: Seq[TreePattern]): Unit = { val plan: LogicalPlan = parser.parsePlan(query) - assert(patterns.exists(plan.containsPattern)) + assert(patterns.exists(plan.containsPattern), s"Failed to parse $query, plan: $plan") assert(plan.containsAnyPattern(UNRESOLVED_RELATION, LOCAL_RELATION)) } def checkPipeSelect(query: String): Unit = check(query, Seq(PROJECT)) checkPipeSelect("TABLE t |> SELECT 1 AS X") checkPipeSelect("TABLE t |> SELECT 1 AS X, 2 AS Y |> SELECT X + Y AS Z") checkPipeSelect("VALUES (0), (1) tab(col) |> SELECT col * 2 AS result") + checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y") + checkPipeSelect("TABLE t |> EXTEND X + 1 AS Y, X + 2 Z") // Basic WHERE operators. def checkPipeWhere(query: String): Unit = check(query, Seq(FILTER)) checkPipeWhere("TABLE t |> WHERE X = 1") diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 71d81b06463f1..2acf25640ef78 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { val sessionHandle = client.openSession(user, "") val infoValue = client.getInfo(sessionHandle, GetInfoType.CLI_ODBC_KEYWORDS) // scalastyle:off line.size.limit - assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") + assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") // scalastyle:on line.size.limit } } From f5bb11cfb3ef983c44ea702ad4b8163a8ad47fb8 Mon Sep 17 00:00:00 2001 From: Neil Ramaswamy Date: Fri, 22 Nov 2024 14:30:51 +0900 Subject: [PATCH 53/58] [SPARK-50301][SS] Make TransformWithState metrics reflect their intuitive meanings ### What changes were proposed in this pull request? These changes make the following changes to metrics in TWS: - `allUpdatesTimeMs` now captures the time it takes to process all the new data with the user's stateful processor. - `timerProcessingTimeMs` was added to capture the time it takes to process all the user's timers. - `allRemovalsTimeMs` now captures the time it takes to do TTL cleanup at the end of a micro-batch. - `commitTimeMs` now captures _only_ the time it takes to commit the state, not the TTL cleanup. With these metrics, a user can have a fairly clear picture of where time is being spent in a micro-batch that uses TWS: ![image](https://github.com/user-attachments/assets/87a0dc9c-c71b-4d55-8623-8970ad83adf6) ### Why are the changes needed? The metrics today misrepresent what they're actually measuring. ### Does this PR introduce _any_ user-facing change? Yes. Metrics for TWS are changing. However, since TWS is `private[sql]`, this shouldn't impact any real users. ### How was this patch tested? We don't have any way to test these metrics in _any_ stateful operator for streaming today. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48862 from neilramaswamy/spark-50301. Authored-by: Neil Ramaswamy Signed-off-by: Jungtaek Lim --- .../streaming/TransformWithStateExec.scala | 39 +++++++---- .../streaming/TransformWithStateSuite.scala | 70 +++++++++++++++++++ 2 files changed, 97 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index 107f98b09f858..f4705b89d5a87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -343,11 +343,20 @@ case class TransformWithStateExec( CompletionIterator[InternalRow, Iterator[InternalRow]] = { val allUpdatesTimeMs = longMetric("allUpdatesTimeMs") val commitTimeMs = longMetric("commitTimeMs") - val timeoutLatencyMs = longMetric("allRemovalsTimeMs") + val timerProcessingTimeMs = longMetric("timerProcessingTimeMs") + // In TWS, allRemovalsTimeMs is the time taken to remove state due to TTL. + // It does not measure any time taken by explicit calls from the user's state processor + // that clear()s state variables. + // + // allRemovalsTimeMs is not granular enough to distinguish between user-caused removals and + // TTL-caused removals. We could leave this empty and have two custom metrics, but leaving + // this as always 0 will be confusing for users. We could also time every call to clear(), but + // that could have performance penalties. So, we choose to capture TTL-only removals. + val allRemovalsTimeMs = longMetric("allRemovalsTimeMs") val currentTimeNs = System.nanoTime val updatesStartTimeNs = currentTimeNs - var timeoutProcessingStartTimeNs = currentTimeNs + var timerProcessingStartTimeNs = currentTimeNs // If timeout is based on event time, then filter late data based on watermark val filteredIter = watermarkPredicateForDataForLateEvents match { @@ -360,9 +369,13 @@ case class TransformWithStateExec( val newDataProcessorIter = CompletionIterator[InternalRow, Iterator[InternalRow]]( processNewData(filteredIter), { - // Once the input is processed, mark the start time for timeout processing to measure + // Note: Due to the iterator lazy execution, this metric also captures the time taken + // by the upstream (consumer) operators in addition to the processing in this operator. + allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs) + + // Once the input is processed, mark the start time for timer processing to measure // it separately from the overall processing time. - timeoutProcessingStartTimeNs = System.nanoTime + timerProcessingStartTimeNs = System.nanoTime processorHandle.setHandleState(StatefulProcessorHandleState.DATA_PROCESSED) }) @@ -376,9 +389,10 @@ case class TransformWithStateExec( private def getIterator(): Iterator[InternalRow] = CompletionIterator[InternalRow, Iterator[InternalRow]]( processTimers(timeMode, processorHandle), { - // Note: `timeoutLatencyMs` also includes the time the parent operator took for - // processing output returned through iterator. - timeoutLatencyMs += NANOSECONDS.toMillis(System.nanoTime - timeoutProcessingStartTimeNs) + // Note: `timerProcessingTimeMs` also includes the time the parent operators take for + // processing output returned from the timers that fire. + timerProcessingTimeMs += + NANOSECONDS.toMillis(System.nanoTime - timerProcessingStartTimeNs) processorHandle.setHandleState(StatefulProcessorHandleState.TIMER_PROCESSED) }) } @@ -387,13 +401,12 @@ case class TransformWithStateExec( // Return an iterator of all the rows generated by all the keys, such that when fully // consumed, all the state updates will be committed by the state store CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, { - // Note: Due to the iterator lazy execution, this metric also captures the time taken - // by the upstream (consumer) operators in addition to the processing in this operator. - allUpdatesTimeMs += NANOSECONDS.toMillis(System.nanoTime - updatesStartTimeNs) + allRemovalsTimeMs += timeTakenMs { + processorHandle.doTtlCleanup() + } + commitTimeMs += timeTakenMs { if (isStreaming) { - // clean up any expired user state - processorHandle.doTtlCleanup() store.commit() } else { store.abort() @@ -419,6 +432,8 @@ case class TransformWithStateExec( StatefulOperatorCustomSumMetric("numMapStateVars", "Number of map state variables"), StatefulOperatorCustomSumMetric("numDeletedStateVars", "Number of deleted state variables"), // metrics around timers + StatefulOperatorCustomSumMetric("timerProcessingTimeMs", + "Number of milliseconds taken to process all timers"), StatefulOperatorCustomSumMetric("numRegisteredTimers", "Number of registered timers"), StatefulOperatorCustomSumMetric("numDeletedTimers", "Number of deleted timers"), StatefulOperatorCustomSumMetric("numExpiredTimers", "Number of expired timers"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 3ef5c57ee3d07..505775d4f6a9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -401,6 +401,30 @@ class StatefulProcessorWithCompositeTypes extends RunningCountStatefulProcessor } } +// For each record, creates a timer to fire in 10 seconds that sleeps for 1 second. +class SleepingTimerProcessor extends StatefulProcessor[String, String, String] { + override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {} + + override def handleInputRows( + key: String, + inputRows: Iterator[String], + timerValues: TimerValues): Iterator[String] = { + inputRows.flatMap { _ => + val currentTime = timerValues.getCurrentProcessingTimeInMs() + getHandle.registerTimer(currentTime + 10000) + None + } + } + + override def handleExpiredTimer( + key: String, + timerValues: TimerValues, + expiredTimerInfo: ExpiredTimerInfo): Iterator[String] = { + Thread.sleep(1000) + Iterator.single(key) + } +} + /** * Class that adds tests for transformWithState stateful streaming operator */ @@ -708,6 +732,52 @@ class TransformWithStateSuite extends StateStoreMetricsTest ) } + test("transformWithState - timer duration should be reflected in metrics") { + val clock = new StreamManualClock + val inputData = MemoryStream[String] + val result = inputData.toDS() + .groupByKey(x => x) + .transformWithState( + new SleepingTimerProcessor, TimeMode.ProcessingTime(), OutputMode.Update()) + + testStream(result, OutputMode.Update())( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, "a"), + AdvanceManualClock(1 * 1000), + // Side effect: timer scheduled for t = 1 + 10 = 11. + CheckNewAnswer(), + Execute { q => + val metrics = q.lastProgress.stateOperators(0).customMetrics + assert(metrics.get("numRegisteredTimers") === 1) + assert(metrics.get("timerProcessingTimeMs") < 2000) + }, + + AddData(inputData, "b"), + AdvanceManualClock(1 * 1000), + // Side effect: timer scheduled for t = 2 + 10 = 12. + CheckNewAnswer(), + Execute { q => + val metrics = q.lastProgress.stateOperators(0).customMetrics + assert(metrics.get("numRegisteredTimers") === 1) + assert(metrics.get("timerProcessingTimeMs") < 2000) + }, + + AddData(inputData, "c"), + // Time is currently 2 and we need to advance past 12. So, advance by 11 seconds. + AdvanceManualClock(11 * 1000), + CheckNewAnswer("a", "b"), + Execute { q => + val metrics = q.lastProgress.stateOperators(0).customMetrics + assert(metrics.get("numRegisteredTimers") === 1) + + // Both timers should have fired and taken 1 second each to process. + assert(metrics.get("timerProcessingTimeMs") >= 2000) + }, + + StopStream + ) + } + test("Use statefulProcessor without transformWithState - handle should be absent") { val processor = new RunningCountStatefulProcessor() val ex = intercept[Exception] { From 8d7e3d4af618116b6a3c306456b2001f2ac13c10 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 22 Nov 2024 15:36:27 +0900 Subject: [PATCH 54/58] Revert "[SPARK-50298][PYTHON][CONNECT] Implement verifySchema parameter of createDataFrame in Spark Connect" This reverts commit e1477a34d7457a1f31164e4e70e00c9912a8de4b. --- python/pyspark/sql/connect/conversion.py | 6 +-- python/pyspark/sql/connect/session.py | 30 +++--------- .../sql/tests/connect/test_parity_arrow.py | 3 +- python/pyspark/sql/tests/test_arrow.py | 46 ++++++++----------- 4 files changed, 31 insertions(+), 54 deletions(-) diff --git a/python/pyspark/sql/connect/conversion.py b/python/pyspark/sql/connect/conversion.py index f689c439f5f68..d803f37c5b9f1 100644 --- a/python/pyspark/sql/connect/conversion.py +++ b/python/pyspark/sql/connect/conversion.py @@ -322,7 +322,7 @@ def convert_other(value: Any) -> Any: return lambda value: value @staticmethod - def convert(data: Sequence[Any], schema: StructType, verifySchema: bool = False) -> "pa.Table": + def convert(data: Sequence[Any], schema: StructType) -> "pa.Table": assert isinstance(data, list) and len(data) > 0 assert schema is not None and isinstance(schema, StructType) @@ -372,8 +372,8 @@ def convert(data: Sequence[Any], schema: StructType, verifySchema: bool = False) ] ) ) - table = pa.Table.from_arrays(pylist, schema=pa_schema) - return table.cast(pa_schema, safe=verifySchema) + + return pa.Table.from_arrays(pylist, schema=pa_schema) class ArrowTableToRowsConversion: diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index e7292bf8804f8..83b0496a84274 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -50,7 +50,6 @@ ) import urllib -from pyspark._globals import _NoValue, _NoValueType from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.dataframe import DataFrame as ParentDataFrame from pyspark.sql.connect.logging import logger @@ -450,7 +449,7 @@ def createDataFrame( data: Union["pd.DataFrame", "np.ndarray", "pa.Table", Iterable[Any]], schema: Optional[Union[AtomicType, StructType, str, List[str], Tuple[str, ...]]] = None, samplingRatio: Optional[float] = None, - verifySchema: Union[_NoValueType, bool] = _NoValue, + verifySchema: Optional[bool] = None, ) -> "ParentDataFrame": assert data is not None if isinstance(data, DataFrame): @@ -462,6 +461,9 @@ def createDataFrame( if samplingRatio is not None: warnings.warn("'samplingRatio' is ignored. It is not supported with Spark Connect.") + if verifySchema is not None: + warnings.warn("'verifySchema' is ignored. It is not supported with Spark Connect.") + _schema: Optional[Union[AtomicType, StructType]] = None _cols: Optional[List[str]] = None _num_cols: Optional[int] = None @@ -574,10 +576,7 @@ def createDataFrame( "spark.sql.session.timeZone", "spark.sql.execution.pandas.convertToArrowArraySafely" ) - if verifySchema is _NoValue: - verifySchema = safecheck == "true" - - ser = ArrowStreamPandasSerializer(cast(str, timezone), verifySchema) + ser = ArrowStreamPandasSerializer(cast(str, timezone), safecheck == "true") _table = pa.Table.from_batches( [ @@ -597,9 +596,6 @@ def createDataFrame( ).cast(arrow_schema) elif isinstance(data, pa.Table): - if verifySchema is _NoValue: - verifySchema = False - prefer_timestamp_ntz = is_timestamp_ntz_preferred() (timezone,) = self._client.get_configs("spark.sql.session.timeZone") @@ -617,10 +613,7 @@ def createDataFrame( _table = ( _check_arrow_table_timestamps_localize(data, schema, True, timezone) - .cast( - to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True), - safe=verifySchema, - ) + .cast(to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True)) .rename_columns(schema.names) ) @@ -659,12 +652,6 @@ def createDataFrame( # The _table should already have the proper column names. _cols = None - if verifySchema is not _NoValue: - warnings.warn( - "'verifySchema' is ignored. It is not supported" - " with np.ndarray input on Spark Connect." - ) - else: _data = list(data) @@ -696,15 +683,12 @@ def createDataFrame( errorClass="CANNOT_DETERMINE_TYPE", messageParameters={} ) - if verifySchema is _NoValue: - verifySchema = True - from pyspark.sql.connect.conversion import LocalDataToArrowConversion # Spark Connect will try its best to build the Arrow table with the # inferred schema in the client side, and then rename the columns and # cast the datatypes in the server side. - _table = LocalDataToArrowConversion.convert(_data, _schema, cast(bool, verifySchema)) + _table = LocalDataToArrowConversion.convert(_data, _schema) # TODO: Beside the validation on number of columns, we should also check # whether the Arrow Schema is compatible with the user provided Schema. diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow.py b/python/pyspark/sql/tests/connect/test_parity_arrow.py index 99d03ad1a4409..d47a367a5460a 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow.py +++ b/python/pyspark/sql/tests/connect/test_parity_arrow.py @@ -137,8 +137,9 @@ def test_toPandas_udt(self): def test_create_dataframe_namedtuples(self): self.check_create_dataframe_namedtuples(True) + @unittest.skip("Spark Connect does not support verifySchema.") def test_createDataFrame_verifySchema(self): - self.check_createDataFrame_verifySchema(True) + super().test_createDataFrame_verifySchema() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 99149d1a23d3e..19d0db9894317 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -533,11 +533,6 @@ def test_createDataFrame_arrow_pandas(self): self.assertEqual(df_arrow.collect(), df_pandas.collect()) def test_createDataFrame_verifySchema(self): - for arrow_enabled in [True, False]: - with self.subTest(arrow_enabled=arrow_enabled): - self.check_createDataFrame_verifySchema(arrow_enabled) - - def check_createDataFrame_verifySchema(self, arrow_enabled): data = {"id": [1, 2, 3], "value": [100000000000, 200000000000, 300000000000]} # data.value should fail schema validation when verifySchema is True schema = StructType( @@ -552,32 +547,29 @@ def check_createDataFrame_verifySchema(self, arrow_enabled): table = pa.table(data) df = self.spark.createDataFrame(table, schema=schema) self.assertEqual(df.collect(), expected) + with self.assertRaises(Exception): self.spark.createDataFrame(table, schema=schema, verifySchema=True) - if arrow_enabled: - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": True}): - # pandas DataFrame with Arrow optimization - pdf = pd.DataFrame(data) + # pandas DataFrame with Arrow optimization + pdf = pd.DataFrame(data) + df = self.spark.createDataFrame(pdf, schema=schema) + # verifySchema defaults to `spark.sql.execution.pandas.convertToArrowArraySafely`, + # which is false by default + self.assertEqual(df.collect(), expected) + with self.assertRaises(Exception): + with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": True}): df = self.spark.createDataFrame(pdf, schema=schema) - # verifySchema defaults to `spark.sql.execution.pandas.convertToArrowArraySafely`, - # which is false by default - self.assertEqual(df.collect(), expected) - with self.assertRaises(Exception): - with self.sql_conf( - {"spark.sql.execution.pandas.convertToArrowArraySafely": True} - ): - df = self.spark.createDataFrame(pdf, schema=schema) - with self.assertRaises(Exception): - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=True) - else: - # pandas DataFrame without Arrow optimization - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): - pdf = pd.DataFrame(data) - with self.assertRaises(Exception): - self.spark.createDataFrame(pdf, schema=schema) # verifySchema defaults to True - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=False) - self.assertEqual(df.collect(), expected) + with self.assertRaises(Exception): + df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=True) + + # pandas DataFrame without Arrow optimization + with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): + pdf = pd.DataFrame(data) + with self.assertRaises(Exception): + df = self.spark.createDataFrame(pdf, schema=schema) # verifySchema defaults to True + df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=False) + self.assertEqual(df.collect(), expected) def _createDataFrame_toggle(self, data, schema=None): with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): From de1c77b9da0198f4e3ce099bea13fcef4a4a8c1f Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 22 Nov 2024 15:36:32 +0900 Subject: [PATCH 55/58] Revert "[SPARK-50291][PYTHON] Standardize verifySchema parameter of createDataFrame in Spark Classic" This reverts commit aea9e875fba49c5dac230c8c1ce5ee0c53fd4fde. --- python/pyspark/sql/pandas/conversion.py | 37 +++++------------- python/pyspark/sql/session.py | 35 +++++++---------- .../sql/tests/connect/test_parity_arrow.py | 4 -- python/pyspark/sql/tests/test_arrow.py | 39 ------------------- .../pyspark/sql/tests/typing/test_session.yml | 13 +++---- 5 files changed, 29 insertions(+), 99 deletions(-) diff --git a/python/pyspark/sql/pandas/conversion.py b/python/pyspark/sql/pandas/conversion.py index 0c612bf4eae3d..172a4fc4b2343 100644 --- a/python/pyspark/sql/pandas/conversion.py +++ b/python/pyspark/sql/pandas/conversion.py @@ -27,7 +27,6 @@ ) from warnings import warn -from pyspark._globals import _NoValue, _NoValueType from pyspark.errors.exceptions.captured import unwrap_spark_exception from pyspark.loose_version import LooseVersion from pyspark.util import _load_from_socket @@ -353,7 +352,7 @@ def createDataFrame( self, data: "PandasDataFrameLike", schema: Union[StructType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> "DataFrame": ... @@ -362,7 +361,7 @@ def createDataFrame( self, data: "pa.Table", schema: Union[StructType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> "DataFrame": ... @@ -371,7 +370,7 @@ def createDataFrame( # type: ignore[misc] data: Union["PandasDataFrameLike", "pa.Table"], schema: Optional[Union[StructType, List[str]]] = None, samplingRatio: Optional[float] = None, - verifySchema: Union[_NoValueType, bool] = _NoValue, + verifySchema: bool = True, ) -> "DataFrame": from pyspark.sql import SparkSession @@ -393,7 +392,7 @@ def createDataFrame( # type: ignore[misc] if schema is None: schema = data.schema.names - return self._create_from_arrow_table(data, schema, timezone, verifySchema) + return self._create_from_arrow_table(data, schema, timezone) # `data` is a PandasDataFrameLike object from pyspark.sql.pandas.utils import require_minimum_pandas_version @@ -406,7 +405,7 @@ def createDataFrame( # type: ignore[misc] if self._jconf.arrowPySparkEnabled() and len(data) > 0: try: - return self._create_from_pandas_with_arrow(data, schema, timezone, verifySchema) + return self._create_from_pandas_with_arrow(data, schema, timezone) except Exception as e: if self._jconf.arrowPySparkFallbackEnabled(): msg = ( @@ -625,11 +624,7 @@ def _get_numpy_record_dtype(self, rec: "np.recarray") -> Optional["np.dtype"]: return np.dtype(record_type_list) if has_rec_fix else None def _create_from_pandas_with_arrow( - self, - pdf: "PandasDataFrameLike", - schema: Union[StructType, List[str]], - timezone: str, - verifySchema: Union[_NoValueType, bool], + self, pdf: "PandasDataFrameLike", schema: Union[StructType, List[str]], timezone: str ) -> "DataFrame": """ Create a DataFrame from a given pandas.DataFrame by slicing it into partitions, converting @@ -662,10 +657,6 @@ def _create_from_pandas_with_arrow( ) import pyarrow as pa - if verifySchema is _NoValue: - # (With Arrow optimization) createDataFrame with `pandas.DataFrame` - verifySchema = self._jconf.arrowSafeTypeConversion() - infer_pandas_dict_as_map = ( str(self.conf.get("spark.sql.execution.pandas.inferPandasDictAsMap")).lower() == "true" ) @@ -734,7 +725,8 @@ def _create_from_pandas_with_arrow( jsparkSession = self._jsparkSession - ser = ArrowStreamPandasSerializer(timezone, verifySchema) + safecheck = self._jconf.arrowSafeTypeConversion() + ser = ArrowStreamPandasSerializer(timezone, safecheck) @no_type_check def reader_func(temp_filename): @@ -753,11 +745,7 @@ def create_iter_server(): return df def _create_from_arrow_table( - self, - table: "pa.Table", - schema: Union[StructType, List[str]], - timezone: str, - verifySchema: Union[_NoValueType, bool], + self, table: "pa.Table", schema: Union[StructType, List[str]], timezone: str ) -> "DataFrame": """ Create a DataFrame from a given pyarrow.Table by slicing it into partitions then @@ -779,10 +767,6 @@ def _create_from_arrow_table( require_minimum_pyarrow_version() - if verifySchema is _NoValue: - # createDataFrame with `pyarrow.Table` - verifySchema = False - prefer_timestamp_ntz = is_timestamp_ntz_preferred() # Create the Spark schema from list of names passed in with Arrow types @@ -802,8 +786,7 @@ def _create_from_arrow_table( schema = from_arrow_schema(table.schema, prefer_timestamp_ntz=prefer_timestamp_ntz) table = _check_arrow_table_timestamps_localize(table, schema, True, timezone).cast( - to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True), - safe=verifySchema, + to_arrow_schema(schema, error_on_duplicated_field_names_in_struct=True) ) # Chunk the Arrow Table into RecordBatches diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 7231d6c10b0b6..e97b844564100 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -38,7 +38,6 @@ TYPE_CHECKING, ) -from pyspark._globals import _NoValue, _NoValueType from pyspark.conf import SparkConf from pyspark.util import is_remote_only from pyspark.sql.conf import RuntimeConfig @@ -1272,7 +1271,7 @@ def createDataFrame( data: Iterable["RowLike"], schema: Union[StructType, str], *, - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1282,7 +1281,7 @@ def createDataFrame( data: "RDD[RowLike]", schema: Union[StructType, str], *, - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1291,7 +1290,7 @@ def createDataFrame( self, data: "RDD[AtomicValue]", schema: Union[AtomicType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1300,7 +1299,7 @@ def createDataFrame( self, data: Iterable["AtomicValue"], schema: Union[AtomicType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1319,7 +1318,7 @@ def createDataFrame( self, data: "PandasDataFrameLike", schema: Union[StructType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1328,7 +1327,7 @@ def createDataFrame( self, data: "pa.Table", schema: Union[StructType, str], - verifySchema: Union[_NoValueType, bool] = ..., + verifySchema: bool = ..., ) -> DataFrame: ... @@ -1337,7 +1336,7 @@ def createDataFrame( # type: ignore[misc] data: Union["RDD[Any]", Iterable[Any], "PandasDataFrameLike", "ArrayLike", "pa.Table"], schema: Optional[Union[AtomicType, StructType, str]] = None, samplingRatio: Optional[float] = None, - verifySchema: Union[_NoValueType, bool] = _NoValue, + verifySchema: bool = True, ) -> DataFrame: """ Creates a :class:`DataFrame` from an :class:`RDD`, a list, a :class:`pandas.DataFrame`, @@ -1381,14 +1380,11 @@ def createDataFrame( # type: ignore[misc] if ``samplingRatio`` is ``None``. This option is effective only when the input is :class:`RDD`. verifySchema : bool, optional - verify data types of every row against schema. - If not provided, createDataFrame with - - pyarrow.Table, verifySchema=False - - pandas.DataFrame with Arrow optimization, verifySchema defaults to - `spark.sql.execution.pandas.convertToArrowArraySafely` - - pandas.DataFrame without Arrow optimization, verifySchema=True - - regular Python instances, verifySchema=True - Arrow optimization is enabled/disabled via `spark.sql.execution.arrow.pyspark.enabled`. + verify data types of every row against schema. Enabled by default. + When the input is :class:`pyarrow.Table` or when the input class is + :class:`pandas.DataFrame` and `spark.sql.execution.arrow.pyspark.enabled` is enabled, + this option is not effective. It follows Arrow type coercion. This option is not + supported with Spark Connect. .. versionadded:: 2.1.0 @@ -1588,13 +1584,8 @@ def _create_dataframe( data: Union["RDD[Any]", Iterable[Any]], schema: Optional[Union[DataType, List[str]]], samplingRatio: Optional[float], - verifySchema: Union[_NoValueType, bool], + verifySchema: bool, ) -> DataFrame: - if verifySchema is _NoValue: - # createDataFrame with regular Python instances - # or (without Arrow optimization) createDataFrame with Pandas DataFrame - verifySchema = True - if isinstance(schema, StructType): verify_func = _make_type_verifier(schema) if verifySchema else lambda _: True diff --git a/python/pyspark/sql/tests/connect/test_parity_arrow.py b/python/pyspark/sql/tests/connect/test_parity_arrow.py index d47a367a5460a..885b3001b1db1 100644 --- a/python/pyspark/sql/tests/connect/test_parity_arrow.py +++ b/python/pyspark/sql/tests/connect/test_parity_arrow.py @@ -137,10 +137,6 @@ def test_toPandas_udt(self): def test_create_dataframe_namedtuples(self): self.check_create_dataframe_namedtuples(True) - @unittest.skip("Spark Connect does not support verifySchema.") - def test_createDataFrame_verifySchema(self): - super().test_createDataFrame_verifySchema() - if __name__ == "__main__": from pyspark.sql.tests.connect.test_parity_arrow import * # noqa: F401 diff --git a/python/pyspark/sql/tests/test_arrow.py b/python/pyspark/sql/tests/test_arrow.py index 19d0db9894317..b71bdb1eece28 100644 --- a/python/pyspark/sql/tests/test_arrow.py +++ b/python/pyspark/sql/tests/test_arrow.py @@ -532,45 +532,6 @@ def test_createDataFrame_arrow_pandas(self): df_pandas = self.spark.createDataFrame(pdf) self.assertEqual(df_arrow.collect(), df_pandas.collect()) - def test_createDataFrame_verifySchema(self): - data = {"id": [1, 2, 3], "value": [100000000000, 200000000000, 300000000000]} - # data.value should fail schema validation when verifySchema is True - schema = StructType( - [StructField("id", IntegerType(), True), StructField("value", IntegerType(), True)] - ) - expected = [ - Row(id=1, value=1215752192), - Row(id=2, value=-1863462912), - Row(id=3, value=-647710720), - ] - # Arrow table - table = pa.table(data) - df = self.spark.createDataFrame(table, schema=schema) - self.assertEqual(df.collect(), expected) - - with self.assertRaises(Exception): - self.spark.createDataFrame(table, schema=schema, verifySchema=True) - - # pandas DataFrame with Arrow optimization - pdf = pd.DataFrame(data) - df = self.spark.createDataFrame(pdf, schema=schema) - # verifySchema defaults to `spark.sql.execution.pandas.convertToArrowArraySafely`, - # which is false by default - self.assertEqual(df.collect(), expected) - with self.assertRaises(Exception): - with self.sql_conf({"spark.sql.execution.pandas.convertToArrowArraySafely": True}): - df = self.spark.createDataFrame(pdf, schema=schema) - with self.assertRaises(Exception): - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=True) - - # pandas DataFrame without Arrow optimization - with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): - pdf = pd.DataFrame(data) - with self.assertRaises(Exception): - df = self.spark.createDataFrame(pdf, schema=schema) # verifySchema defaults to True - df = self.spark.createDataFrame(pdf, schema=schema, verifySchema=False) - self.assertEqual(df.collect(), expected) - def _createDataFrame_toggle(self, data, schema=None): with self.sql_conf({"spark.sql.execution.arrow.pyspark.enabled": False}): df_no_arrow = self.spark.createDataFrame(data, schema=schema) diff --git a/python/pyspark/sql/tests/typing/test_session.yml b/python/pyspark/sql/tests/typing/test_session.yml index 98587458efe8a..d6eee82a7678e 100644 --- a/python/pyspark/sql/tests/typing/test_session.yml +++ b/python/pyspark/sql/tests/typing/test_session.yml @@ -17,7 +17,6 @@ - case: createDataFrameStructsValid main: | - from pyspark._globals import _NoValueType from pyspark.sql import SparkSession from pyspark.sql.types import StructType, StructField, StringType, IntegerType @@ -79,14 +78,14 @@ main:18: note: Possible overload variants: main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: Iterable[RowLike], schema: Union[list[str], tuple[str, ...]] = ..., samplingRatio: Optional[float] = ...) -> DataFrame main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: RDD[RowLike], schema: Union[list[str], tuple[str, ...]] = ..., samplingRatio: Optional[float] = ...) -> DataFrame - main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: Iterable[RowLike], schema: Union[StructType, str], *, verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame - main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: RDD[RowLike], schema: Union[StructType, str], *, verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame - main:18: note: def [AtomicValue in (datetime, date, Decimal, bool, str, int, float)] createDataFrame(self, data: RDD[AtomicValue], schema: Union[AtomicType, str], verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame - main:18: note: def [AtomicValue in (datetime, date, Decimal, bool, str, int, float)] createDataFrame(self, data: Iterable[AtomicValue], schema: Union[AtomicType, str], verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame + main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: Iterable[RowLike], schema: Union[StructType, str], *, verifySchema: bool = ...) -> DataFrame + main:18: note: def [RowLike in (list[Any], tuple[Any, ...], Row)] createDataFrame(self, data: RDD[RowLike], schema: Union[StructType, str], *, verifySchema: bool = ...) -> DataFrame + main:18: note: def [AtomicValue in (datetime, date, Decimal, bool, str, int, float)] createDataFrame(self, data: RDD[AtomicValue], schema: Union[AtomicType, str], verifySchema: bool = ...) -> DataFrame + main:18: note: def [AtomicValue in (datetime, date, Decimal, bool, str, int, float)] createDataFrame(self, data: Iterable[AtomicValue], schema: Union[AtomicType, str], verifySchema: bool = ...) -> DataFrame main:18: note: def createDataFrame(self, data: DataFrame, samplingRatio: Optional[float] = ...) -> DataFrame main:18: note: def createDataFrame(self, data: Any, samplingRatio: Optional[float] = ...) -> DataFrame - main:18: note: def createDataFrame(self, data: DataFrame, schema: Union[StructType, str], verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame - main:18: note: def createDataFrame(self, data: Any, schema: Union[StructType, str], verifySchema: Union[_NoValueType, bool] = ...) -> DataFrame + main:18: note: def createDataFrame(self, data: DataFrame, schema: Union[StructType, str], verifySchema: bool = ...) -> DataFrame + main:18: note: def createDataFrame(self, data: Any, schema: Union[StructType, str], verifySchema: bool = ...) -> DataFrame - case: createDataFrameFromEmptyRdd main: | From 5e076efd751040a71bbceca04270d5f2af7158c3 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 22 Nov 2024 15:42:15 +0900 Subject: [PATCH 56/58] [SPARK-50390][PYTHON][DOCS] Refine the docstring for datetime functions - part 2 ### What changes were proposed in this pull request? Refine the docstring for datetime functions ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? updated doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48929 from zhengruifeng/py_doc_7. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 468 +++++++++++++++++------- 1 file changed, 328 insertions(+), 140 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index f5ac8b2c45599..d2873a388617e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -10343,6 +10343,7 @@ def date_add(start: "ColumnOrName", days: Union["ColumnOrName", int]) -> Column: :meth:`pyspark.sql.functions.date_sub` :meth:`pyspark.sql.functions.datediff` :meth:`pyspark.sql.functions.date_diff` + :meth:`pyspark.sql.functions.timestamp_add` Examples -------- @@ -10570,6 +10571,7 @@ def date_diff(end: "ColumnOrName", start: "ColumnOrName") -> Column: :meth:`pyspark.sql.functions.date_add` :meth:`pyspark.sql.functions.date_sub` :meth:`pyspark.sql.functions.datediff` + :meth:`pyspark.sql.functions.timestamp_diff` Examples -------- @@ -10612,6 +10614,7 @@ def date_from_unix_date(days: "ColumnOrName") -> Column: See Also -------- :meth:`pyspark.sql.functions.from_unixtime` + :meth:`pyspark.sql.functions.unix_date` Examples -------- @@ -10764,9 +10767,9 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name input column of values to convert. - format: str, optional + format: literal string, optional format to use to convert date values. Returns @@ -10774,15 +10777,28 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: :class:`~pyspark.sql.Column` date value as :class:`pyspark.sql.types.DateType` type. + See Also + -------- + :meth:`pyspark.sql.functions.to_timestamp` + :meth:`pyspark.sql.functions.try_to_timestamp` + Examples -------- - >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) - >>> df.select(to_date(df.t).alias('date')).collect() - [Row(date=datetime.date(1997, 2, 28))] + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['ts']) + >>> df.select('*', sf.to_date(df.ts)).show() + +-------------------+-----------+ + | ts|to_date(ts)| + +-------------------+-----------+ + |1997-02-28 10:30:00| 1997-02-28| + +-------------------+-----------+ - >>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t']) - >>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect() - [Row(date=datetime.date(1997, 2, 28))] + >>> df.select('*', sf.to_date('ts', 'yyyy-MM-dd HH:mm:ss')).show() + +-------------------+--------------------------------+ + | ts|to_date(ts, yyyy-MM-dd HH:mm:ss)| + +-------------------+--------------------------------+ + |1997-02-28 10:30:00| 1997-02-28| + +-------------------+--------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -10798,12 +10814,37 @@ def unix_date(col: "ColumnOrName") -> Column: .. versionadded:: 3.5.0 + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + input column of values to convert. + + Returns + ------- + :class:`~pyspark.sql.Column` + the number of days since 1970-01-01. + + See Also + -------- + :meth:`pyspark.sql.functions.date_from_unix_date` + :meth:`pyspark.sql.functions.unix_seconds` + :meth:`pyspark.sql.functions.unix_millis` + :meth:`pyspark.sql.functions.unix_micros` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([('1970-01-02',)], ['t']) - >>> df.select(unix_date(to_date(df.t)).alias('n')).collect() - [Row(n=1)] + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('1970-01-02',), ('2022-01-02',)], ['dt']) + >>> df.select('*', sf.unix_date(sf.to_date('dt'))).show() + +----------+----------------------+ + | dt|unix_date(to_date(dt))| + +----------+----------------------+ + |1970-01-02| 1| + |2022-01-02| 18994| + +----------+----------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("unix_date", col) @@ -10815,12 +10856,37 @@ def unix_micros(col: "ColumnOrName") -> Column: .. versionadded:: 3.5.0 + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + input column of values to convert. + + Returns + ------- + :class:`~pyspark.sql.Column` + the number of microseconds since 1970-01-01 00:00:00 UTC. + + See Also + -------- + :meth:`pyspark.sql.functions.unix_date` + :meth:`pyspark.sql.functions.unix_seconds` + :meth:`pyspark.sql.functions.unix_millis` + :meth:`pyspark.sql.functions.timestamp_micros` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([('2015-07-22 10:00:00',)], ['t']) - >>> df.select(unix_micros(to_timestamp(df.t)).alias('n')).collect() - [Row(n=1437584400000000)] + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-07-22 10:00:00',), ('2022-10-09 11:12:13',)], ['ts']) + >>> df.select('*', sf.unix_micros(sf.to_timestamp('ts'))).show() + +-------------------+-----------------------------+ + | ts|unix_micros(to_timestamp(ts))| + +-------------------+-----------------------------+ + |2015-07-22 10:00:00| 1437584400000000| + |2022-10-09 11:12:13| 1665339133000000| + +-------------------+-----------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("unix_micros", col) @@ -10833,12 +10899,37 @@ def unix_millis(col: "ColumnOrName") -> Column: .. versionadded:: 3.5.0 + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + input column of values to convert. + + Returns + ------- + :class:`~pyspark.sql.Column` + the number of milliseconds since 1970-01-01 00:00:00 UTC. + + See Also + -------- + :meth:`pyspark.sql.functions.unix_date` + :meth:`pyspark.sql.functions.unix_seconds` + :meth:`pyspark.sql.functions.unix_micros` + :meth:`pyspark.sql.functions.timestamp_millis` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([('2015-07-22 10:00:00',)], ['t']) - >>> df.select(unix_millis(to_timestamp(df.t)).alias('n')).collect() - [Row(n=1437584400000)] + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-07-22 10:00:00',), ('2022-10-09 11:12:13',)], ['ts']) + >>> df.select('*', sf.unix_millis(sf.to_timestamp('ts'))).show() + +-------------------+-----------------------------+ + | ts|unix_millis(to_timestamp(ts))| + +-------------------+-----------------------------+ + |2015-07-22 10:00:00| 1437584400000| + |2022-10-09 11:12:13| 1665339133000| + +-------------------+-----------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("unix_millis", col) @@ -10851,12 +10942,38 @@ def unix_seconds(col: "ColumnOrName") -> Column: .. versionadded:: 3.5.0 + Parameters + ---------- + col : :class:`~pyspark.sql.Column` or column name + input column of values to convert. + + Returns + ------- + :class:`~pyspark.sql.Column` + the number of seconds since 1970-01-01 00:00:00 UTC. + + See Also + -------- + :meth:`pyspark.sql.functions.unix_date` + :meth:`pyspark.sql.functions.unix_millis` + :meth:`pyspark.sql.functions.unix_micros` + :meth:`pyspark.sql.functions.from_unixtime` + :meth:`pyspark.sql.functions.timestamp_seconds` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> df = spark.createDataFrame([('2015-07-22 10:00:00',)], ['t']) - >>> df.select(unix_seconds(to_timestamp(df.t)).alias('n')).collect() - [Row(n=1437584400)] + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([('2015-07-22 10:00:00',), ('2022-10-09 11:12:13',)], ['ts']) + >>> df.select('*', sf.unix_seconds(sf.to_timestamp('ts'))).show() + +-------------------+------------------------------+ + | ts|unix_seconds(to_timestamp(ts))| + +-------------------+------------------------------+ + |2015-07-22 10:00:00| 1437584400| + |2022-10-09 11:12:13| 1665339133| + +-------------------+------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("unix_seconds", col) @@ -10888,9 +11005,9 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column values to convert. - format: str, optional + format: literal string, optional format to use to convert timestamp values. Returns @@ -10900,6 +11017,7 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: See Also -------- + :meth:`pyspark.sql.functions.to_date` :meth:`pyspark.sql.functions.try_to_timestamp` Examples @@ -10945,13 +11063,14 @@ def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = Non Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name column values to convert. - format: str, optional + format: literal string, optional format to use to convert timestamp values. See Also -------- + :meth:`pyspark.sql.functions.to_date` :meth:`pyspark.sql.functions.to_timestamp` Examples @@ -11365,9 +11484,9 @@ def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss" Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name column of unix time values. - format : str, optional + format : literal string, optional format to use to convert to (default: yyyy-MM-dd HH:mm:ss) Returns @@ -11378,13 +11497,21 @@ def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss" See Also -------- :meth:`pyspark.sql.functions.date_from_unix_date` + :meth:`pyspark.sql.functions.unix_seconds` Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - >>> time_df = spark.createDataFrame([(1428476400,)], ['unix_time']) - >>> time_df.select(from_unixtime('unix_time').alias('ts')).collect() - [Row(ts='2015-04-08 00:00:00')] + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(1428476400,)], ['unix_time']) + >>> df.select('*', sf.from_unixtime('unix_time')).show() + +----------+---------------------------------------------+ + | unix_time|from_unixtime(unix_time, yyyy-MM-dd HH:mm:ss)| + +----------+---------------------------------------------+ + |1428476400| 2015-04-08 00:00:00| + +----------+---------------------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ from pyspark.sql.classic.column import _to_java_column @@ -11420,9 +11547,9 @@ def unix_timestamp( Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str, optional + timestamp : :class:`~pyspark.sql.Column` or column name, optional timestamps of string values. - format : str, optional + format : literal string, optional alternative format to use for converting (default: yyyy-MM-dd HH:mm:ss). Returns @@ -11437,8 +11564,7 @@ def unix_timestamp( Example 1: Returns the current timestamp in UNIX. >>> import pyspark.sql.functions as sf - >>> spark.range(1).select(sf.unix_timestamp().alias('unix_time')).show() - ... # doctest: +SKIP + >>> spark.range(1).select(sf.unix_timestamp()).show() # doctest: +SKIP +----------+ | unix_time| +----------+ @@ -11448,24 +11574,24 @@ def unix_timestamp( Example 2: Using default format 'yyyy-MM-dd HH:mm:ss' parses the timestamp string. >>> import pyspark.sql.functions as sf - >>> time_df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['dt']) - >>> time_df.select(sf.unix_timestamp('dt').alias('unix_time')).show() - +----------+ - | unix_time| - +----------+ - |1428520332| - +----------+ + >>> df = spark.createDataFrame([('2015-04-08 12:12:12',)], ['ts']) + >>> df.select('*', sf.unix_timestamp('ts')).show() + +-------------------+---------------------------------------+ + | ts|unix_timestamp(ts, yyyy-MM-dd HH:mm:ss)| + +-------------------+---------------------------------------+ + |2015-04-08 12:12:12| 1428520332| + +-------------------+---------------------------------------+ Example 3: Using user-specified format 'yyyy-MM-dd' parses the timestamp string. >>> import pyspark.sql.functions as sf - >>> time_df = spark.createDataFrame([('2015-04-08',)], ['dt']) - >>> time_df.select(sf.unix_timestamp('dt', 'yyyy-MM-dd').alias('unix_time')).show() - +----------+ - | unix_time| - +----------+ - |1428476400| - +----------+ + >>> df = spark.createDataFrame([('2015-04-08',)], ['dt']) + >>> df.select('*', sf.unix_timestamp('dt', 'yyyy-MM-dd')).show() + +----------+------------------------------+ + | dt|unix_timestamp(dt, yyyy-MM-dd)| + +----------+------------------------------+ + |2015-04-08| 1428476400| + +----------+------------------------------+ >>> spark.conf.unset("spark.sql.session.timeZone") """ @@ -11499,9 +11625,9 @@ def from_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Col Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name the column that contains timestamps - tz : :class:`~pyspark.sql.Column` or str + tz : :class:`~pyspark.sql.Column` or literal string A string detailing the time zone ID that the input should be adjusted to. It should be in the format of either region-based zone IDs or zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in @@ -11517,13 +11643,27 @@ def from_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Col :class:`~pyspark.sql.Column` timestamp value represented in given timezone. + See Also + -------- + :meth:`pyspark.sql.functions.to_utc_timestamp` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz']) - >>> df.select(from_utc_timestamp(df.ts, "PST").alias('local_time')).collect() - [Row(local_time=datetime.datetime(1997, 2, 28, 2, 30))] - >>> df.select(from_utc_timestamp(df.ts, df.tz).alias('local_time')).collect() - [Row(local_time=datetime.datetime(1997, 2, 28, 19, 30))] + >>> df.select('*', sf.from_utc_timestamp('ts', 'PST')).show() + +-------------------+---+---------------------------+ + | ts| tz|from_utc_timestamp(ts, PST)| + +-------------------+---+---------------------------+ + |1997-02-28 10:30:00|JST| 1997-02-28 02:30:00| + +-------------------+---+---------------------------+ + + >>> df.select('*', sf.from_utc_timestamp(df.ts, df.tz)).show() + +-------------------+---+--------------------------+ + | ts| tz|from_utc_timestamp(ts, tz)| + +-------------------+---+--------------------------+ + |1997-02-28 10:30:00|JST| 1997-02-28 19:30:00| + +-------------------+---+--------------------------+ """ return _invoke_function_over_columns("from_utc_timestamp", timestamp, lit(tz)) @@ -11551,9 +11691,9 @@ def to_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Colum Parameters ---------- - timestamp : :class:`~pyspark.sql.Column` or str + timestamp : :class:`~pyspark.sql.Column` or column name the column that contains timestamps - tz : :class:`~pyspark.sql.Column` or str + tz : :class:`~pyspark.sql.Column` or literal string A string detailing the time zone ID that the input should be adjusted to. It should be in the format of either region-based zone IDs or zone offsets. Region IDs must have the form 'area/city', such as 'America/Los_Angeles'. Zone offsets must be in @@ -11569,13 +11709,27 @@ def to_utc_timestamp(timestamp: "ColumnOrName", tz: Union[Column, str]) -> Colum :class:`~pyspark.sql.Column` timestamp value represented in UTC timezone. + See Also + -------- + :meth:`pyspark.sql.functions.from_utc_timestamp` + Examples -------- + >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz']) - >>> df.select(to_utc_timestamp(df.ts, "PST").alias('utc_time')).collect() - [Row(utc_time=datetime.datetime(1997, 2, 28, 18, 30))] - >>> df.select(to_utc_timestamp(df.ts, df.tz).alias('utc_time')).collect() - [Row(utc_time=datetime.datetime(1997, 2, 28, 1, 30))] + >>> df.select('*', sf.to_utc_timestamp('ts', "PST")).show() + +-------------------+---+-------------------------+ + | ts| tz|to_utc_timestamp(ts, PST)| + +-------------------+---+-------------------------+ + |1997-02-28 10:30:00|JST| 1997-02-28 18:30:00| + +-------------------+---+-------------------------+ + + >>> df.select('*', sf.to_utc_timestamp(df.ts, df.tz)).show() + +-------------------+---+------------------------+ + | ts| tz|to_utc_timestamp(ts, tz)| + +-------------------+---+------------------------+ + |1997-02-28 10:30:00|JST| 1997-02-28 01:30:00| + +-------------------+---+------------------------+ """ return _invoke_function_over_columns("to_utc_timestamp", timestamp, lit(tz)) @@ -11593,7 +11747,7 @@ def timestamp_seconds(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name unix time values. Returns @@ -11601,20 +11755,26 @@ def timestamp_seconds(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` converted timestamp value. + See Also + -------- + :meth:`pyspark.sql.functions.timestamp_millis` + :meth:`pyspark.sql.functions.timestamp_micros` + :meth:`pyspark.sql.functions.unix_seconds` + Examples -------- - >>> from pyspark.sql.functions import timestamp_seconds >>> spark.conf.set("spark.sql.session.timeZone", "UTC") - >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) - >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).show() - +-------------------+ - | ts| - +-------------------+ - |2008-12-25 15:30:00| - +-------------------+ - >>> time_df.select(timestamp_seconds('unix_time').alias('ts')).printSchema() - root - |-- ts: timestamp (nullable = true) + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(1230219000,), (1280219000,)], ['seconds']) + >>> df.select('*', sf.timestamp_seconds('seconds')).show() + +----------+--------------------------+ + | seconds|timestamp_seconds(seconds)| + +----------+--------------------------+ + |1230219000| 2008-12-25 15:30:00| + |1280219000| 2010-07-27 08:23:20| + +----------+--------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ @@ -11630,7 +11790,7 @@ def timestamp_millis(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name unix time values. Returns @@ -11638,19 +11798,26 @@ def timestamp_millis(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` converted timestamp value. + See Also + -------- + :meth:`pyspark.sql.functions.timestamp_seconds` + :meth:`pyspark.sql.functions.timestamp_micros` + :meth:`pyspark.sql.functions.unix_millis` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "UTC") - >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) - >>> time_df.select(timestamp_millis(time_df.unix_time).alias('ts')).show() - +-------------------+ - | ts| - +-------------------+ - |1970-01-15 05:43:39| - +-------------------+ - >>> time_df.select(timestamp_millis('unix_time').alias('ts')).printSchema() - root - |-- ts: timestamp (nullable = true) + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(1230219000,), (1280219000,)], ['millis']) + >>> df.select('*', sf.timestamp_millis('millis')).show() + +----------+------------------------+ + | millis|timestamp_millis(millis)| + +----------+------------------------+ + |1230219000| 1970-01-15 05:43:39| + |1280219000| 1970-01-15 19:36:59| + +----------+------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("timestamp_millis", col) @@ -11665,7 +11832,7 @@ def timestamp_micros(col: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or column name unix time values. Returns @@ -11673,19 +11840,26 @@ def timestamp_micros(col: "ColumnOrName") -> Column: :class:`~pyspark.sql.Column` converted timestamp value. + See Also + -------- + :meth:`pyspark.sql.functions.timestamp_seconds` + :meth:`pyspark.sql.functions.timestamp_millis` + :meth:`pyspark.sql.functions.unix_micros` + Examples -------- >>> spark.conf.set("spark.sql.session.timeZone", "UTC") - >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) - >>> time_df.select(timestamp_micros(time_df.unix_time).alias('ts')).show() - +--------------------+ - | ts| - +--------------------+ - |1970-01-01 00:20:...| - +--------------------+ - >>> time_df.select(timestamp_micros('unix_time').alias('ts')).printSchema() - root - |-- ts: timestamp (nullable = true) + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(1230219000,), (1280219000,)], ['micros']) + >>> df.select('*', sf.timestamp_micros('micros')).show(truncate=False) + +----------+------------------------+ + |micros |timestamp_micros(micros)| + +----------+------------------------+ + |1230219000|1970-01-01 00:20:30.219 | + |1280219000|1970-01-01 00:21:20.219 | + +----------+------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ return _invoke_function_over_columns("timestamp_micros", col) @@ -11701,13 +11875,13 @@ def timestamp_diff(unit: str, start: "ColumnOrName", end: "ColumnOrName") -> Col Parameters ---------- - unit : str + unit : literal string This indicates the units of the difference between the given timestamps. Supported options are (case insensitive): "YEAR", "QUARTER", "MONTH", "WEEK", "DAY", "HOUR", "MINUTE", "SECOND", "MILLISECOND" and "MICROSECOND". - start : :class:`~pyspark.sql.Column` or str + start : :class:`~pyspark.sql.Column` or column name A timestamp which the expression subtracts from `endTimestamp`. - end : :class:`~pyspark.sql.Column` or str + end : :class:`~pyspark.sql.Column` or column name A timestamp from which the expression subtracts `startTimestamp`. Returns @@ -11715,31 +11889,38 @@ def timestamp_diff(unit: str, start: "ColumnOrName", end: "ColumnOrName") -> Col :class:`~pyspark.sql.Column` the difference between the timestamps. + See Also + -------- + :meth:`pyspark.sql.functions.datediff` + :meth:`pyspark.sql.functions.date_diff` + Examples -------- >>> import datetime >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame( ... [(datetime.datetime(2016, 3, 11, 9, 0, 7), datetime.datetime(2024, 4, 2, 9, 0, 7))], - ... ).toDF("start", "end") - >>> df.select(sf.timestamp_diff("year", "start", "end")).show() - +-------------------------------+ - |timestampdiff(year, start, end)| - +-------------------------------+ - | 8| - +-------------------------------+ - >>> df.select(sf.timestamp_diff("WEEK", "start", "end")).show() - +-------------------------------+ - |timestampdiff(WEEK, start, end)| - +-------------------------------+ - | 420| - +-------------------------------+ - >>> df.select(sf.timestamp_diff("day", "end", "start")).show() - +------------------------------+ - |timestampdiff(day, end, start)| - +------------------------------+ - | -2944| - +------------------------------+ + ... ['ts1', 'ts2']) + >>> df.select('*', sf.timestamp_diff('year', 'ts1', 'ts2')).show() + +-------------------+-------------------+-----------------------------+ + | ts1| ts2|timestampdiff(year, ts1, ts2)| + +-------------------+-------------------+-----------------------------+ + |2016-03-11 09:00:07|2024-04-02 09:00:07| 8| + +-------------------+-------------------+-----------------------------+ + + >>> df.select('*', sf.timestamp_diff('WEEK', 'ts1', 'ts2')).show() + +-------------------+-------------------+-----------------------------+ + | ts1| ts2|timestampdiff(WEEK, ts1, ts2)| + +-------------------+-------------------+-----------------------------+ + |2016-03-11 09:00:07|2024-04-02 09:00:07| 420| + +-------------------+-------------------+-----------------------------+ + + >>> df.select('*', sf.timestamp_diff('day', df.ts2, df.ts1)).show() + +-------------------+-------------------+----------------------------+ + | ts1| ts2|timestampdiff(day, ts2, ts1)| + +-------------------+-------------------+----------------------------+ + |2016-03-11 09:00:07|2024-04-02 09:00:07| -2944| + +-------------------+-------------------+----------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -11761,13 +11942,13 @@ def timestamp_add(unit: str, quantity: "ColumnOrName", ts: "ColumnOrName") -> Co Parameters ---------- - unit : str + unit : literal string This indicates the units of the difference between the given timestamps. Supported options are (case insensitive): "YEAR", "QUARTER", "MONTH", "WEEK", "DAY", "HOUR", "MINUTE", "SECOND", "MILLISECOND" and "MICROSECOND". - quantity : :class:`~pyspark.sql.Column` or str + quantity : :class:`~pyspark.sql.Column` or column name The number of units of time that you want to add. - ts : :class:`~pyspark.sql.Column` or str + ts : :class:`~pyspark.sql.Column` or column name A timestamp to which you want to add. Returns @@ -11775,34 +11956,41 @@ def timestamp_add(unit: str, quantity: "ColumnOrName", ts: "ColumnOrName") -> Co :class:`~pyspark.sql.Column` the difference between the timestamps. + See Also + -------- + :meth:`pyspark.sql.functions.dateadd` + :meth:`pyspark.sql.functions.date_add` + Examples -------- >>> import datetime >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame( ... [(datetime.datetime(2016, 3, 11, 9, 0, 7), 2), - ... (datetime.datetime(2024, 4, 2, 9, 0, 7), 3)], ["ts", "quantity"]) - >>> df.select(sf.timestamp_add("year", "quantity", "ts")).show() - +--------------------------------+ - |timestampadd(year, quantity, ts)| - +--------------------------------+ - | 2018-03-11 09:00:07| - | 2027-04-02 09:00:07| - +--------------------------------+ - >>> df.select(sf.timestamp_add("WEEK", sf.lit(5), "ts")).show() - +-------------------------+ - |timestampadd(WEEK, 5, ts)| - +-------------------------+ - | 2016-04-15 09:00:07| - | 2024-05-07 09:00:07| - +-------------------------+ - >>> df.select(sf.timestamp_add("day", sf.lit(-5), "ts")).show() - +-------------------------+ - |timestampadd(day, -5, ts)| - +-------------------------+ - | 2016-03-06 09:00:07| - | 2024-03-28 09:00:07| - +-------------------------+ + ... (datetime.datetime(2024, 4, 2, 9, 0, 7), 3)], ['ts', 'quantity']) + >>> df.select('*', sf.timestamp_add('year', 'quantity', 'ts')).show() + +-------------------+--------+--------------------------------+ + | ts|quantity|timestampadd(year, quantity, ts)| + +-------------------+--------+--------------------------------+ + |2016-03-11 09:00:07| 2| 2018-03-11 09:00:07| + |2024-04-02 09:00:07| 3| 2027-04-02 09:00:07| + +-------------------+--------+--------------------------------+ + + >>> df.select('*', sf.timestamp_add('WEEK', sf.lit(5), df.ts)).show() + +-------------------+--------+-------------------------+ + | ts|quantity|timestampadd(WEEK, 5, ts)| + +-------------------+--------+-------------------------+ + |2016-03-11 09:00:07| 2| 2016-04-15 09:00:07| + |2024-04-02 09:00:07| 3| 2024-05-07 09:00:07| + +-------------------+--------+-------------------------+ + + >>> df.select('*', sf.timestamp_add('day', sf.lit(-5), 'ts')).show() + +-------------------+--------+-------------------------+ + | ts|quantity|timestampadd(day, -5, ts)| + +-------------------+--------+-------------------------+ + |2016-03-11 09:00:07| 2| 2016-03-06 09:00:07| + |2024-04-02 09:00:07| 3| 2024-03-28 09:00:07| + +-------------------+--------+-------------------------+ """ from pyspark.sql.classic.column import _to_java_column From f9d2f4226366d442b0433b4ad0c6cbf17950cc1c Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 22 Nov 2024 16:20:56 +0900 Subject: [PATCH 57/58] [SPARK-50394][PYTHON][INFRA] Reduce parallelism in Pure Python library builds ### What changes were proposed in this pull request? This PR proposes to decrease parallelism in Pure Python library builds ### Why are the changes needed? In order to make the tests more robust: https://github.com/apache/spark/actions/workflows/build_python_connect.yml https://github.com/apache/spark/actions/workflows/build_python_connect35.yml Now they fail because of OOM. ### Does this PR introduce _any_ user-facing change? No, test-only ### How was this patch tested? Will monitor the build: https://github.com/apache/spark/actions/workflows/build_python_connect.yml https://github.com/apache/spark/actions/workflows/build_python_connect35.yml ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48932 from HyukjinKwon/reduce-parallelism. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_python_connect.yml | 2 +- .github/workflows/build_python_connect35.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index f668d813ef26e..d57a0c2b91623 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -93,7 +93,7 @@ jobs: # Several tests related to catalog requires to run them sequencially, e.g., writing a table in a listener. ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-connect,pyspark-ml-connect # None of tests are dependent on each other in Pandas API on Spark so run them in parallel - ./python/run-tests --parallelism=4 --python-executables=python3 --modules pyspark-pandas-connect-part0,pyspark-pandas-connect-part1,pyspark-pandas-connect-part2,pyspark-pandas-connect-part3 + ./python/run-tests --parallelism=2 --python-executables=python3 --modules pyspark-pandas-connect-part0,pyspark-pandas-connect-part1,pyspark-pandas-connect-part2,pyspark-pandas-connect-part3 # Stop Spark Connect server. ./sbin/stop-connect-server.sh diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index 276c9ad9d2f92..4b7a6b82b9527 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -98,7 +98,7 @@ jobs: # Run branch-3.5 tests ./python/run-tests --parallelism=1 --python-executables=python3 --modules pyspark-connect # None of tests are dependent on each other in Pandas API on Spark so run them in parallel - ./python/run-tests --parallelism=4 --python-executables=python3 --modules pyspark-pandas-connect,pyspark-pandas-slow-connect + ./python/run-tests --parallelism=2 --python-executables=python3 --modules pyspark-pandas-connect,pyspark-pandas-slow-connect - name: Upload test results to report if: always() uses: actions/upload-artifact@v4 From 77055b8125591abb6bff9cb029bb725ad7d47d2e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 22 Nov 2024 15:54:13 +0800 Subject: [PATCH 58/58] [SPARK-50385][CORE] Use class name prefix for REST Submission API thread pool ### What changes were proposed in this pull request? This PR aims to use a meaningful class name prefix for REST Submission API thread pool instead of the default value of Jetty QueuedThreadPool, `"qtp"+super.hashCode()`. https://github.com/dekellum/jetty/blob/3dc0120d573816de7d6a83e2d6a97035288bdd4a/jetty-util/src/main/java/org/eclipse/jetty/util/thread/QueuedThreadPool.java#L64 ### Why are the changes needed? This is helpful during JVM investigation. **BEFORE (4.0.0-preview2)** ``` $ SPARK_MASTER_OPTS='-Dspark.master.rest.enabled=true' sbin/start-master.sh $ jstack 28217 | grep qtp "qtp1925630411-52" #52 daemon prio=5 os_prio=31 cpu=0.07ms elapsed=19.06s tid=0x0000000134906c10 nid=0xde03 runnable [0x0000000314592000] "qtp1925630411-53" #53 daemon prio=5 os_prio=31 cpu=0.05ms elapsed=19.06s tid=0x0000000134ac6810 nid=0xc603 runnable [0x000000031479e000] "qtp1925630411-54" #54 daemon prio=5 os_prio=31 cpu=0.06ms elapsed=19.06s tid=0x000000013491ae10 nid=0xdc03 runnable [0x00000003149aa000] "qtp1925630411-55" #55 daemon prio=5 os_prio=31 cpu=0.08ms elapsed=19.06s tid=0x0000000134ac9810 nid=0xc803 runnable [0x0000000314bb6000] "qtp1925630411-56" #56 daemon prio=5 os_prio=31 cpu=0.04ms elapsed=19.06s tid=0x0000000134ac9e10 nid=0xda03 runnable [0x0000000314dc2000] "qtp1925630411-57" #57 daemon prio=5 os_prio=31 cpu=0.05ms elapsed=19.06s tid=0x0000000134aca410 nid=0xca03 runnable [0x0000000314fce000] "qtp1925630411-58" #58 daemon prio=5 os_prio=31 cpu=0.04ms elapsed=19.06s tid=0x0000000134acaa10 nid=0xcb03 runnable [0x00000003151da000] "qtp1925630411-59" #59 daemon prio=5 os_prio=31 cpu=0.06ms elapsed=19.06s tid=0x0000000134acb010 nid=0xcc03 runnable [0x00000003153e6000] "qtp1925630411-60-acceptor-0108e9815-ServerConnector1e497474{HTTP/1.1, (http/1.1)}{M3-Max.local:6066}" #60 daemon prio=3 os_prio=31 cpu=0.11ms elapsed=19.06s tid=0x00000001317ffa10 nid=0xcd03 runnable [0x00000003155f2000] "qtp1925630411-61-acceptor-11d90f2aa-ServerConnector1e497474{HTTP/1.1, (http/1.1)}{M3-Max.local:6066}" #61 daemon prio=3 os_prio=31 cpu=0.10ms elapsed=19.06s tid=0x00000001314ed610 nid=0xcf03 waiting on condition [0x00000003157fe000] ``` **AFTER** ``` $ SPARK_MASTER_OPTS='-Dspark.master.rest.enabled=true' sbin/start-master.sh $ jstack 28317 | grep StandaloneRestServer "StandaloneRestServer-52" #52 daemon prio=5 os_prio=31 cpu=0.09ms elapsed=60.06s tid=0x00000001284a8e10 nid=0xdb03 runnable [0x000000032cfce000] "StandaloneRestServer-53" #53 daemon prio=5 os_prio=31 cpu=0.06ms elapsed=60.06s tid=0x00000001284acc10 nid=0xda03 runnable [0x000000032d1da000] "StandaloneRestServer-54" #54 daemon prio=5 os_prio=31 cpu=0.05ms elapsed=60.06s tid=0x00000001284ae610 nid=0xd803 runnable [0x000000032d3e6000] "StandaloneRestServer-55" #55 daemon prio=5 os_prio=31 cpu=0.09ms elapsed=60.06s tid=0x00000001284aec10 nid=0xd703 runnable [0x000000032d5f2000] "StandaloneRestServer-56" #56 daemon prio=5 os_prio=31 cpu=0.06ms elapsed=60.06s tid=0x00000001284af210 nid=0xc803 runnable [0x000000032d7fe000] "StandaloneRestServer-57" #57 daemon prio=5 os_prio=31 cpu=0.05ms elapsed=60.06s tid=0x00000001284af810 nid=0xc903 runnable [0x000000032da0a000] "StandaloneRestServer-58" #58 daemon prio=5 os_prio=31 cpu=0.06ms elapsed=60.06s tid=0x00000001284afe10 nid=0xcb03 runnable [0x000000032dc16000] "StandaloneRestServer-59" #59 daemon prio=5 os_prio=31 cpu=0.05ms elapsed=60.06s tid=0x00000001284b0410 nid=0xcc03 runnable [0x000000032de22000] "StandaloneRestServer-60-acceptor-04aefbaa8-ServerConnector44284d85{HTTP/1.1, (http/1.1)}{M3-Max.local:6066}" #60 daemon prio=3 os_prio=31 cpu=0.13ms elapsed=60.05s tid=0x000000015cda1a10 nid=0xcd03 runnable [0x000000032e02e000] "StandaloneRestServer-61-acceptor-148976251-ServerConnector44284d85{HTTP/1.1, (http/1.1)}{M3-Max.local:6066}" #61 daemon prio=3 os_prio=31 cpu=0.12ms elapsed=60.05s tid=0x000000015cd1c810 nid=0xce03 waiting on condition [0x000000032e23a000] ``` ### Does this PR introduce _any_ user-facing change? No, the thread names are accessed during the debugging. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48924 from dongjoon-hyun/SPARK-50385. Authored-by: Dongjoon Hyun Signed-off-by: panbingkun --- .../org/apache/spark/deploy/rest/RestSubmissionServer.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index 877349da18dd0..9e3aab125689f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -94,6 +94,7 @@ private[spark] abstract class RestSubmissionServer( */ private def doStart(startPort: Int): (Server, Int) = { val threadPool = new QueuedThreadPool(masterConf.get(MASTER_REST_SERVER_MAX_THREADS)) + threadPool.setName(getClass().getSimpleName()) if (Utils.isJavaVersionAtLeast21 && masterConf.get(MASTER_REST_SERVER_VIRTUAL_THREADS)) { val newVirtualThreadPerTaskExecutor = classOf[Executors].getMethod("newVirtualThreadPerTaskExecutor")